You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by nchammas <gi...@git.apache.org> on 2014/08/03 01:32:02 UTC

[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

GitHub user nchammas opened a pull request:

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

    [SPARK-2627] have the build enforce PEP 8 automatically

    As described in [SPARK-2627](https://issues.apache.org/jira/browse/SPARK-2627), we'd like Python code to automatically be checked for PEP 8 compliance by Jenkins. This pull request aims to do that.
    
    Notes:
    * We may need to install [`pep8`](https://pypi.python.org/pypi/pep8) on the build server.
    * I'm expecting tests to fail now that PEP 8 compliance is being checked as part of the build. I'm fine with cleaning up any remaining PEP 8 violations as part of this pull request.
    * I did not understand why the RAT and scalastyle reports are saved to text files. I did the same for the PEP 8 check, but only so that the console output style can match those for the RAT and scalastyle checks. The PEP 8 report is removed right after the check is complete.
    * Updates to the ["Contributing to Spark"](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) guide will be submitted elsewhere, as I don't believe that text is part of the Spark repo.

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

    $ git pull https://github.com/nchammas/spark master

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

    https://github.com/apache/spark/pull/1744.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 #1744
    
----
commit 75ad55211397345e5192a19513d478b913028506
Author: Nicholas Chammas <ni...@gmail.com>
Date:   2014-08-02T23:13:21Z

    make check output style consistent
    
    The RAT and PEP8 checks don’t print a blank line after successful runs.
    The scalastyle check shouldn’t either.

commit 61c07b911b033b0356326397a6b7168dc94e6632
Author: Nicholas Chammas <ni...@gmail.com>
Date:   2014-08-02T23:15:49Z

    [SPARK-2627] add Python linter
    
    This guy just runs the pep8 utility on all code in the python
    directory, minus cloudpickle, which is a 3rd-party library.

commit 12440faa1a0a8b7eb41c5778ea49c360cb0de532
Author: Nicholas Chammas <ni...@gmail.com>
Date:   2014-08-02T23:23:06Z

    [SPARK-2627] add Scala linter
    
    This guy just calls scalastyle.

commit 0541ebb5f30973aa54157256b4e43decfa9de8ba
Author: Nicholas Chammas <ni...@gmail.com>
Date:   2014-08-02T23:23:30Z

    [SPARK-2627] call Python linter from run-tests

commit 723ed39fb5067e3827a3e1f5434fb111b8e498cc
Author: Nicholas Chammas <ni...@gmail.com>
Date:   2014-08-02T23:24:00Z

    always delete the report file

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51388412
  
    And branch-1.1 too.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15893160
  
    --- Diff: python/pyspark/tests.py ---
    @@ -62,53 +62,53 @@ def setUp(self):
             self.N = 1 << 16
             self.l = [i for i in xrange(self.N)]
             self.data = zip(self.l, self.l)
    -        self.agg = Aggregator(lambda x: [x], 
    -                lambda x, y: x.append(y) or x,
    -                lambda x, y: x.extend(y) or x)
    +        self.agg = Aggregator(lambda x: [x],
    +                              lambda x, y: x.append(y) or x,
    +                              lambda x, y: x.extend(y) or x)
     
         def test_in_memory(self):
             m = InMemoryMerger(self.agg)
             m.mergeValues(self.data)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = InMemoryMerger(self.agg)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_small_dataset(self):
             m = ExternalMerger(self.agg, 1000)
             m.mergeValues(self.data)
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 1000)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_medium_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeValues(self.data)
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)) * 3)
    +                         sum(xrange(self.N)) * 3)
     
         def test_huge_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)),
    -                self.N * 10)
    +                         self.N * 10)
    --- End diff --
    
    Oh, I see, thanks. I should read pep8 more carefully.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51000620
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17821/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50999124
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17821/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50983745
  
    Hey Pat!
    
    1. I've edited the title accordingly.
    2. Makes sense. I'll take a crack at fetching `pep8` lazily as you describe. Is there something you can point me to that I can use as a reference for how to do this?
    
    By the way, I'm not sure I resolved my merge conflicts in the best way. It looks like changes I merged into my fork from upstream got re-included in this PR.
    
    Is there a quick way to fix this? Apologies, I'm still learning how to use git properly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892897
  
    --- Diff: python/pyspark/tests.py ---
    @@ -622,14 +624,15 @@ def test_newhadoop(self):
                 valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect())
             self.assertEqual(result, array_data)
     
    -        conf = {"mapreduce.outputformat.class" :
    -                     "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
    -                 "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable",
    -                 "mapred.output.value.class" : "org.apache.spark.api.python.DoubleArrayWritable",
    -                 "mapred.output.dir" : basepath + "/newdataset/"}
    -        self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(conf,
    +        conf = {"mapreduce.outputformat.class":
    +                "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
    +                "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
    +                "mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
    +                "mapred.output.dir": basepath + "/newdataset/"}
    --- End diff --
    
    I'll fix this to follow the style you specified, but note that the first pair in the `dict` is longer than 100 characters so it will have to be broken up.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51127066
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17877/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51135706
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15891925
  
    --- Diff: python/pyspark/tests.py ---
    @@ -685,30 +688,32 @@ def test_reserialization(self):
             result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect())
             self.assertEqual(result1, data)
     
    -        rdd.saveAsHadoopFile(basepath + "/reserialize/hadoop",
    -                             "org.apache.hadoop.mapred.SequenceFileOutputFormat")
    +        rdd.saveAsHadoopFile(
    +            basepath + "/reserialize/hadoop",
    +            "org.apache.hadoop.mapred.SequenceFileOutputFormat")
             result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect())
             self.assertEqual(result2, data)
     
    -        rdd.saveAsNewAPIHadoopFile(basepath + "/reserialize/newhadoop",
    -                             "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    +        rdd.saveAsNewAPIHadoopFile(
    +            basepath + "/reserialize/newhadoop",
    +            "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    --- End diff --
    
    This is different with others, what is the rule of indent of arguments?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51135662
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17884/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51202476
  
    Jenkins! Wake up and retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51273423
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17955/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51380803
  
    I've made one new commit to address @davies's comments.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51001740
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17826/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51372597
  
    One question (maybe too late), could we customize the lint tool to relax some rules? Such as change line width to 100, or change indent to 2 spaces. Recently, I found that it's not easy to write the code to fit with 4 spaces and 80 chars, always needed to break the line, and finally, the code became not easy to read, especially when you have several levels of structures, such as class, function, function, if and so on.
    
    If we can have a tool to format the code to follow pep8 that will even better. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51127238
  
    This Jenkins failure is my fault (see #1623).  I'll have Jenkins re-run this as soon as we've committed  the fix.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50985019
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17810/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51377103
  
    > Oh, sorry, I saw it break some lines which does not have more than 100 characters.
    
    Ah yeah, that happened because of a mistake I made when I called `autopep8` on some files with the default setting of breaking lines at 80 characters. I tried to undo all those changes. You [found one](https://github.com/apache/spark/pull/1744#discussion-diff-15891820) that I missed, but it should be fixed now.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51263913
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51263511
  
    @rxin @pwendell I'm not sure what to do here. Jenkins appears to have a very bad hangover.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15891820
  
    --- Diff: python/pyspark/tests.py ---
    @@ -993,15 +1008,17 @@ def test_serialize(self):
     
     @unittest.skipIf(not _have_numpy, "NumPy not installed")
     class NumPyTests(PySparkTestCase):
    +
         """General PySpark tests that depend on numpy """
     
         def test_statcounter_array(self):
    -        x = self.sc.parallelize([np.array([1.0,1.0]), np.array([2.0,2.0]), np.array([3.0,3.0])])
    --- End diff --
    
    this line has 96 characters, need break?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50983935
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17809/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

Posted by pwendell <gi...@git.apache.org>.
Github user pwendell commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50982162
  
    Hey nick - thanks for taking a crack at this. It's great to see us adding more automated code quality checks. Couple things:
    
    1. Could you add `[PySpark]` to the title of this PR? We are using tags like that to do sorting amongst the committership and it will get noticed that way.
    2. In terms of the dependency on pep8, we've tried really hard to avoid having exogenous dependencies in Spark. It makes porting things like our QA environment very difficult. So one idea - could this have a script that just lazily fetches the pep8 library directly? For instance, this is what we do with our sbt tool - we just wget the sbt jar... it seems like you could do something similar for pep8. Not sure if that totally works, but just an idea.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51143491
  
    I don't see how these test failures might be related to the changes introduced in this PR. I see that the issue @JoshRosen called out earlier here has been [resolved](https://github.com/apache/spark/pull/1771), so that can't be it.
    
    More confusingly, the report ends with this:
    ```
    [info] All tests passed.
    [info] Passed: Total 797, Failed 0, Errors 0, Passed 797, Ignored 7
    [error] (streaming-flume/test:test) sbt.TestsFailedException: Tests unsuccessful
    [error] Total time: 3157 s, completed Aug 4, 2014 7:30:52 PM
    ```
    
    @rxin - Any pointers?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51268200
  
    Jenkins is just _done_ with this PR. Like, _"dude, get out of my life"_ kind of done.
    
    @rxin What should I do? Close this and open a new PR?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15784129
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    Ah, it's not haphazard. This is squarely my fault. `autopep8` defaults to a max length of 79.
    
        --max-line-length n   set maximum allowed line length (default: 79)
    
    I'll fix this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51222571
  
    There's something wrong with [the build](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17940/consoleFull).
    
    > [infoBuild timed out (after 120 minutes). Marking the build as failed.
    Build was aborted
    Recording test results
    Finished: FAILURE
    
    I also noticed that [earlier failures](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17903/consoleFull) were related to some broken Flume tests, which seems to be a known (but yet unresolved) issue: [SPARK-2798 - Jenkins build failing due to missing scalatest in flume-sink module](https://issues.apache.org/jira/browse/SPARK-2798).
    
    How shall we proceed here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892997
  
    --- Diff: python/pyspark/tests.py ---
    @@ -62,53 +62,53 @@ def setUp(self):
             self.N = 1 << 16
             self.l = [i for i in xrange(self.N)]
             self.data = zip(self.l, self.l)
    -        self.agg = Aggregator(lambda x: [x], 
    -                lambda x, y: x.append(y) or x,
    -                lambda x, y: x.extend(y) or x)
    +        self.agg = Aggregator(lambda x: [x],
    +                              lambda x, y: x.append(y) or x,
    +                              lambda x, y: x.extend(y) or x)
     
         def test_in_memory(self):
             m = InMemoryMerger(self.agg)
             m.mergeValues(self.data)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = InMemoryMerger(self.agg)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_small_dataset(self):
             m = ExternalMerger(self.agg, 1000)
             m.mergeValues(self.data)
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 1000)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_medium_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeValues(self.data)
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)) * 3)
    +                         sum(xrange(self.N)) * 3)
     
         def test_huge_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)),
    -                self.N * 10)
    +                         self.N * 10)
    --- End diff --
    
    The old idententation yields the following `pep8` error:
    
    ```
    ./python/pyspark/tests.py:111:17: E128 continuation line under-indented for visual indent
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51119767
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17877/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51003309
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17826/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892109
  
    --- Diff: python/pyspark/tests.py ---
    @@ -622,14 +624,15 @@ def test_newhadoop(self):
                 valueConverter="org.apache.spark.api.python.WritableToDoubleArrayConverter").collect())
             self.assertEqual(result, array_data)
     
    -        conf = {"mapreduce.outputformat.class" :
    -                     "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
    -                 "mapred.output.key.class" : "org.apache.hadoop.io.IntWritable",
    -                 "mapred.output.value.class" : "org.apache.spark.api.python.DoubleArrayWritable",
    -                 "mapred.output.dir" : basepath + "/newdataset/"}
    -        self.sc.parallelize(array_data).saveAsNewAPIHadoopDataset(conf,
    +        conf = {"mapreduce.outputformat.class":
    +                "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat",
    +                "mapred.output.key.class": "org.apache.hadoop.io.IntWritable",
    +                "mapred.output.value.class": "org.apache.spark.api.python.DoubleArrayWritable",
    +                "mapred.output.dir": basepath + "/newdataset/"}
    --- End diff --
    
    If the dict crosses multiple lines, it's better to write as:
    
    d = {
        "key": value,
    } 
    
    also, each pair should end with ","


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51367511
  
    By the way, if there is something majorly wrong with this PR (e.g. it's too big; I took the wrong approach; etc.) I am more than happy to scrap it and start over, or otherwise redo large parts of it as required.
    
    One way or the other, I'd like to see this through to the finish.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51376388
  
    > Maybe it's not enforced by pep8, just changed to make the code looks better?
    
    The only changes that should be like that are the ones related to the indentation of some of the `newAPIHadoop...()` calls in `tests.py`, and I believe there are only a handful of them. 
    
    The remainder of the whitespace changes are indeed mandated by PEP 8.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51374986
  
    Oh, sorry, I saw it break some lines which does not have more than 100 characters.
    
    LTGM now, but i'm have some questions about the changes in this PR. Maybe it's not enforced by pep8, just changed to make the code looks better?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50999759
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17816/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15782338
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    This line doesn't look too long, but it's not a big deal to wrap 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51263909
  
    He's probably en route somewhere because he woke up early this morning to catch a flight...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51138990
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17896/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51136814
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17887/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51131720
  
    QA tests have started for PR 1744. This patch DID NOT merge cleanly! <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17884/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51263924
  
    Jenkins, add to whitelist.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51273180
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51267122
  
    Jenkins, you good-looking devil, you, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51087364
  
    @rxin @pwendell This PR is ready for review.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15894002
  
    --- Diff: python/pyspark/tests.py ---
    @@ -993,15 +1008,17 @@ def test_serialize(self):
     
     @unittest.skipIf(not _have_numpy, "NumPy not installed")
     class NumPyTests(PySparkTestCase):
    +
         """General PySpark tests that depend on numpy """
     
         def test_statcounter_array(self):
    -        x = self.sc.parallelize([np.array([1.0,1.0]), np.array([2.0,2.0]), np.array([3.0,3.0])])
    --- End diff --
    
    ... and it's now 99 characters (as opposed to the original 96) because PEP 8 wants you to put spaces after your commas:
    
    ```
    ./python/pyspark/tests.py:1018:47: E231 missing whitespace after ','
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51118631
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17873/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51202770
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17940/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51361188
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18026/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51259645
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892226
  
    --- Diff: python/pyspark/tests.py ---
    @@ -62,53 +62,53 @@ def setUp(self):
             self.N = 1 << 16
             self.l = [i for i in xrange(self.N)]
             self.data = zip(self.l, self.l)
    -        self.agg = Aggregator(lambda x: [x], 
    -                lambda x, y: x.append(y) or x,
    -                lambda x, y: x.extend(y) or x)
    +        self.agg = Aggregator(lambda x: [x],
    +                              lambda x, y: x.append(y) or x,
    +                              lambda x, y: x.extend(y) or x)
     
         def test_in_memory(self):
             m = InMemoryMerger(self.agg)
             m.mergeValues(self.data)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = InMemoryMerger(self.agg)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_small_dataset(self):
             m = ExternalMerger(self.agg, 1000)
             m.mergeValues(self.data)
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 1000)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data))
             self.assertEqual(m.spills, 0)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
         def test_medium_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeValues(self.data)
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)))
    +                         sum(xrange(self.N)))
     
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(sum(v) for k, v in m.iteritems()),
    -                sum(xrange(self.N)) * 3)
    +                         sum(xrange(self.N)) * 3)
     
         def test_huge_dataset(self):
             m = ExternalMerger(self.agg, 10)
             m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10))
             self.assertTrue(m.spills >= 1)
             self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)),
    -                self.N * 10)
    +                         self.N * 10)
    --- End diff --
    
    why?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51139738
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17903/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51203116
  
    See Reynold, Jenkins is smarter than you think.
    
    He also seems to work on East Coast time. :P


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51381274
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18038/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15893054
  
    --- Diff: python/pyspark/tests.py ---
    @@ -685,30 +688,32 @@ def test_reserialization(self):
             result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect())
             self.assertEqual(result1, data)
     
    -        rdd.saveAsHadoopFile(basepath + "/reserialize/hadoop",
    -                             "org.apache.hadoop.mapred.SequenceFileOutputFormat")
    +        rdd.saveAsHadoopFile(
    +            basepath + "/reserialize/hadoop",
    +            "org.apache.hadoop.mapred.SequenceFileOutputFormat")
             result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect())
             self.assertEqual(result2, data)
     
    -        rdd.saveAsNewAPIHadoopFile(basepath + "/reserialize/newhadoop",
    -                             "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    +        rdd.saveAsNewAPIHadoopFile(
    +            basepath + "/reserialize/newhadoop",
    +            "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    --- End diff --
    
    Sorry for the comment, this is really part of pep8. Your changes are correct.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132083
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17886/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132626
  
    Lord, keeping up with the changes is tough. Jenkins, do me a favor mang and retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892643
  
    --- Diff: python/pyspark/tests.py ---
    @@ -993,15 +1008,17 @@ def test_serialize(self):
     
     @unittest.skipIf(not _have_numpy, "NumPy not installed")
     class NumPyTests(PySparkTestCase):
    +
         """General PySpark tests that depend on numpy """
     
         def test_statcounter_array(self):
    -        x = self.sc.parallelize([np.array([1.0,1.0]), np.array([2.0,2.0]), np.array([3.0,3.0])])
    --- End diff --
    
    Ah, no need for the break. I'll fix this. (It's actually 99 characters without the break, I believe.)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51134952
  
    OK, I've submitted the final set of fixes to undo the damage caused by calling `autopep8` with the wrong settings. Once tests pass for this, I'd say it's ready for review.
    
    In a separate PR I'd like to expand the PEP 8 checks to cover the whole project and not just the `python` directory. Right now, the EC2 scripts are not checked, for example. But for now I want to get this PR through as quickly as possible to minimize the catch-up work I have to do to merge in new changes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50983857
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17808/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50981650
  
    QA tests have started for PR 1744. This patch DID NOT merge cleanly! <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17800/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51380827
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51367875
  
    This looks good to me. What do you think, @JoshRosen / @davies ?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51135881
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17896/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132147
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17886/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15892740
  
    --- Diff: python/pyspark/tests.py ---
    @@ -685,30 +688,32 @@ def test_reserialization(self):
             result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect())
             self.assertEqual(result1, data)
     
    -        rdd.saveAsHadoopFile(basepath + "/reserialize/hadoop",
    -                             "org.apache.hadoop.mapred.SequenceFileOutputFormat")
    +        rdd.saveAsHadoopFile(
    +            basepath + "/reserialize/hadoop",
    +            "org.apache.hadoop.mapred.SequenceFileOutputFormat")
             result2 = sorted(self.sc.sequenceFile(basepath + "/reserialize/hadoop").collect())
             self.assertEqual(result2, data)
     
    -        rdd.saveAsNewAPIHadoopFile(basepath + "/reserialize/newhadoop",
    -                             "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    +        rdd.saveAsNewAPIHadoopFile(
    +            basepath + "/reserialize/newhadoop",
    +            "org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat")
    --- End diff --
    
    I must admit for these changes to the indenting of `saveAsNewAPIHadoopFile` I just tried to make it consistent within the file. Is there a more specific rule I should follow?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50984858
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17809/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50977706
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17788/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51164849
  
    Maybe it was started before that fix got merged. Let's run this again,


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50998146
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17816/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51374590
  
    @davies The line length to check for is [set to 100 in `tox.ini`](https://github.com/nchammas/spark/blob/983d963a7600ce61caa230984461e91b343b0d33/tox.ini#L17), and `pep8` checks there for configuration settings.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by JoshRosen <gi...@git.apache.org>.
Github user JoshRosen commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15782822
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    In general, it looks like a bunch of these changes are related to line length, but `autopep8` seems to have done it fairly haphazardly.  I think 100-character lines are probably fine, since that seems to be the standard that we've been using.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15782962
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    Yeah, I assumed `autopep8` would only introduce wraps at the 100 character mark. I'll take a closer look at the changes it made and fix any such wraps that are unnecessary. If you find more yourself please add a note and I'll be glad to fix them.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51118760
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17873/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50998139
  
    Patrick, I believe I've addressed the issues you called out. I'm not sure if the approach I took to getting `pep8` at runtime is the best. Let me know what you think.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132720
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50982524
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17800/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132704
  
    Haha, I don't know if it responds to such complex phrases. 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51164855
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51273157
  
    I think Jenkins is sick ... we can keep trying ...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by davies <gi...@git.apache.org>.
Github user davies commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51376336
  
    LGTM. no more comments.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51001657
  
    Jenkins, my man, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15782487
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    Hmm, I'll look into it. It makes the comment block look weird. I let `autopep8` have its way with some of the files, so that may be 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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51387885
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18038/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15783885
  
    --- Diff: python/pyspark/__init__.py ---
    @@ -53,7 +53,8 @@
     # mllib that depend on top level pyspark packages, which transitively depend on python's random.
     # Since Python's import logic looks for modules in the current package first, we eliminate
     # mllib.random as a candidate for C{import random} by removing the first search path, the script's
    -# location, in order to force the loader to look in Python's top-level modules for C{random}.
    +# location, in order to force the loader to look in Python's top-level
    --- End diff --
    
    Taking a closer look, you're right. `autopep8` seems to have broken lines quite haphazardly indeed. :( My apologies. I will fix all occurrences of this problem.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15780210
  
    --- Diff: dev/lint-python ---
    @@ -0,0 +1,63 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +
    +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
    +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)"
    +PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt"
    +
    +cd $SPARK_ROOT_DIR
    +
    +# Get pep8 at runtime so that we don't rely on it being installed on the build server.
    +#+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162
    +#+ TODOs:
    +#+  - Dynamically determine latest release version of pep8 and use that.
    +#+  - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?))
    +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py"
    +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py"
    +
    +curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH"    
    +curl_status=$?
    +
    +if [ $curl_status -ne 0 ]
    +    then
    +        echo "Failed to download pep8.py from \"$PEP8_SCRIPT_REMOTE_PATH\"."
    +        exit $curl_status
    +fi
    +
    +
    +# There is no need to write this output to a file
    +#+ first, but we do so so that the check status can
    +#+ be output before the report, like with the
    +#+ scalastyle and RAT checks.
    +python $PEP8_SCRIPT_PATH ./python --exclude="cloudpickle.py" \
    +    > "$PEP8_REPORT_PATH"
    +pep8_status=${PIPESTATUS[0]} #$?
    +
    +if [ $pep8_status -ne 0 ]
    +    then
    --- End diff --
    
    here too


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51353319
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/18026/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] have the build enforce PEP 8 auto...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50977688
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17788/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51133444
  
    lol, it was worth a shot.
    
    Anyway, I just noticed that there are a few line break fixes I missed. Gonna submit one more commit for that. Sorry about the spam.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51388197
  
    Ok I'm merging this in master. Thanks, @nchammas.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51280200
  
    Alright, now that Jenkins is sober again, I'd say this PR is ready for another review. @rxin @pwendell @JoshRosen


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51139547
  
    Jenkins, retest this please.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/1744#discussion_r15780205
  
    --- Diff: dev/lint-python ---
    @@ -0,0 +1,63 @@
    +#!/usr/bin/env bash
    +
    +#
    +# 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.
    +#
    +
    +SCRIPT_DIR="$( cd "$( dirname "$0" )" && pwd )"
    +SPARK_ROOT_DIR="$(dirname $SCRIPT_DIR)"
    +PEP8_REPORT_PATH="$SPARK_ROOT_DIR/dev/pep8-report.txt"
    +
    +cd $SPARK_ROOT_DIR
    +
    +# Get pep8 at runtime so that we don't rely on it being installed on the build server.
    +#+ See: https://github.com/apache/spark/pull/1744#issuecomment-50982162
    +#+ TODOs:
    +#+  - Dynamically determine latest release version of pep8 and use that.
    +#+  - Download this from a more reliable source. (GitHub raw can be flaky, apparently. (?))
    +PEP8_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pep8.py"
    +PEP8_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/jcrocholl/pep8/1.5.7/pep8.py"
    +
    +curl --silent -o "$PEP8_SCRIPT_PATH" "$PEP8_SCRIPT_REMOTE_PATH"    
    +curl_status=$?
    +
    +if [ $curl_status -ne 0 ]
    --- End diff --
    
    can we follow the rest of spark bash style to put then on the same line as the if? we can also reduce the level of indication there


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51277715
  
    QA results for PR 1744:<br>- This patch PASSES unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17955/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51143047
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17903/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50983867
  
    QA results for PR 1744:<br>- This patch FAILED unit tests.<br>- This patch merges cleanly<br>- This patch adds no public classes<br><br>For more information see test ouptut:<br>https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17808/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by rxin <gi...@git.apache.org>.
Github user rxin commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51373160
  
    The line length is already 100 in this PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51139536
  
    Hmm, I'm not sure why this latest round of tests failed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51132863
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17887/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-50984092
  
    QA tests have started for PR 1744. This patch merges cleanly. <br>View progress: https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/17810/consoleFull


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] spark pull request: [SPARK-2627] [PySpark] have the build enforce ...

Posted by nchammas <gi...@git.apache.org>.
Github user nchammas commented on the pull request:

    https://github.com/apache/spark/pull/1744#issuecomment-51393145
  
    Thank you @davies @pwendell @JoshRosen and @rxin for reviewing this PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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