You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by ashwinshankar77 <gi...@git.apache.org> on 2015/10/08 22:42:14 UTC

[GitHub] spark pull request: [SPARK-8170][Python]Add signal handler to trap...

GitHub user ashwinshankar77 opened a pull request:

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

    [SPARK-8170][Python]Add signal handler to trap Ctrl-C in pyspark and cancel all running jobs

    This patch adds a signal handler to trap Ctrl-C and cancels running job.

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

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

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

    https://github.com/apache/spark/pull/9033.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 #9033
    
----
commit a77b32a43406c23bdd8d3e307a19c8bf2284b2bc
Author: Ashwin Shankar <as...@netflix.com>
Date:   2015-10-08T20:19:57Z

    Add signal to trap Ctrl-C in pyspark and cancel all running jobs

----


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147462187
  
    @ashwinshankar77 I worried that the failed test is actually related to the change in this PR, could you run test locally?


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146716840
  
    LGTM


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147460194
  
    Hi @davies, did you have any comments on the patch ? Also is it possible for a user
    to kick off Jenkins build for his/her patch, when such unrelated test failures happen ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146717125
  
      [Test build #1863 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1863/consoleFull) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146969358
  
      [Test build #1868 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1868/console) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `case class BinaryHashJoinNode(`
      * `case class BroadcastHashJoinNode(`
      * `trait HashJoinNode `



---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146719675
  
      [Test build #1863 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1863/console) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HasWeightCol(Params):`
      * `class IsotonicRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredictionCol,`
      * `class IsotonicRegressionModel(JavaModel):`
      * `case class Average(child: Expression) extends DeclarativeAggregate `
      * `case class Count(child: Expression) extends DeclarativeAggregate `
      * `case class First(child: Expression) extends DeclarativeAggregate `
      * `case class Last(child: Expression) extends DeclarativeAggregate `
      * `case class Max(child: Expression) extends DeclarativeAggregate `
      * `case class Min(child: Expression) extends DeclarativeAggregate `
      * `abstract class StddevAgg(child: Expression) extends DeclarativeAggregate `
      * `case class Sum(child: Expression) extends DeclarativeAggregate `
      * `sealed abstract class AggregateFunction2 extends Expression with ImplicitCastInputTypes `
      * `abstract class ImperativeAggregate extends AggregateFunction2 `
      * `case class GeneratedExpressionCode(var code: String, var isNull: String, var value: String)`
      * `case class RoundRobinPartitioning(numPartitions: Int) extends Partitioning `
      * `case class Coalesce(numPartitions: Int, child: SparkPlan) extends UnaryNode `
      * `case class BinaryHashJoinNode(`
      * `case class BroadcastHashJoinNode(`
      * `trait HashJoinNode `



---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146680651
  
    Can one of the admins verify this patch?


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146679774
  
    @davies 
    @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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146692696
  
    Jenkins, OK to test


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146965408
  
      [Test build #1868 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1868/consoleFull) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147466132
  
    @davies I did run it locally and the pyspark tests passed. I also ran ./dev/run-tests and tests passed. :
    Running PySpark tests. Output is in /Users/ashankar/Documents/workspace/spark-apache/spark/python/unit-tests.log
    Will test against the following Python executables: ['python2.6']
    Will test the following Python modules: ['pyspark-core', 'pyspark-ml', 'pyspark-mllib', 'pyspark-sql', 'pyspark-streaming']
    Finished test(python2.6): pyspark.conf (4s)
    Finished test(python2.6): pyspark.broadcast (6s)
    Finished test(python2.6): pyspark.accumulators (9s)
    Finished test(python2.6): pyspark.rdd (16s)
    Finished test(python2.6): pyspark.shuffle (0s)
    Finished test(python2.6): pyspark.serializers (12s)
    Finished test(python2.6): pyspark.profiler (5s)
    Finished test(python2.6): pyspark.context (21s)
    Finished test(python2.6): pyspark.ml.clustering (10s)
    Finished test(python2.6): pyspark.ml.feature (16s)
    Finished test(python2.6): pyspark.ml.classification (16s)
    Finished test(python2.6): pyspark.ml.recommendation (17s)
    Finished test(python2.6): pyspark.ml.regression (16s)
    Finished test(python2.6): pyspark.ml.tuning (16s)
    Finished test(python2.6): pyspark.ml.evaluation (11s)
    Finished test(python2.6): pyspark.ml.tests (16s)
    Finished test(python2.6): pyspark.mllib.classification (18s)
    Finished test(python2.6): pyspark.mllib.evaluation (10s)
    Finished test(python2.6): pyspark.mllib.feature (16s)
    Finished test(python2.6): pyspark.mllib.linalg.__init__ (0s)
    Finished test(python2.6): pyspark.mllib.clustering (25s)
    Finished test(python2.6): pyspark.mllib.fpm (11s)
    Finished test(python2.6): pyspark.mllib.random (8s)
    Finished test(python2.6): pyspark.mllib.linalg.distributed (14s)
    Finished test(python2.6): pyspark.mllib.recommendation (19s)
    Finished test(python2.6): pyspark.mllib.stat.KernelDensity (0s)
    Finished test(python2.6): pyspark.mllib.stat._statistics (12s)
    Finished test(python2.6): pyspark.mllib.regression (19s)
    Finished test(python2.6): pyspark.mllib.util (9s)
    Finished test(python2.6): pyspark.mllib.tree (15s)
    Finished test(python2.6): pyspark.sql.types (8s)
    Finished test(python2.6): pyspark.sql.context (15s)
    Finished test(python2.6): pyspark.sql.column (13s)
    Finished test(python2.6): pyspark.sql.group (15s)
    Finished test(python2.6): pyspark.sql.dataframe (29s)
    Finished test(python2.6): pyspark.tests (156s)
    Finished test(python2.6): pyspark.sql.window (5s)
    Finished test(python2.6): pyspark.sql.functions (19s)
    Finished test(python2.6): pyspark.streaming.util (0s)
    Finished test(python2.6): pyspark.sql.readwriter (23s)
    Finished test(python2.6): pyspark.sql.tests (41s)
    Finished test(python2.6): pyspark.mllib.tests (135s)
    Finished test(python2.6): pyspark.streaming.tests (371s)
    Tests passed in 552 seconds


---
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-8170][Python]Add signal handler to trap...

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

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


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147462038
  
      [Test build #1878 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1878/consoleFull) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146970969
  
    Test failures are unrelated


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147470337
  
      [Test build #1883 has started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1883/consoleFull) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).


---
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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147479481
  
      [Test build #1883 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1883/console) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-147471963
  
      [Test build #1878 has finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1878/console) for   PR 9033 at commit [`a77b32a`](https://github.com/apache/spark/commit/a77b32a43406c23bdd8d3e307a19c8bf2284b2bc).
     * This patch **passes all tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-8170][Python]Add signal handler to trap...

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

    https://github.com/apache/spark/pull/9033#issuecomment-146964773
  
    Test failures are unrelated.


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