You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by rxin <gi...@git.apache.org> on 2015/02/03 07:17:33 UTC

[GitHub] spark pull request: [SPARK-5549] Define TaskContext interface in S...

GitHub user rxin opened a pull request:

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

    [SPARK-5549] Define TaskContext interface in Scala.

    So the interface documentation shows up in ScalaDoc.

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

    $ git pull https://github.com/rxin/spark TaskContext-scala

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

    https://github.com/apache/spark/pull/4324.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 #4324
    
----
commit 87dd537e591af257007f86a600e39c97b87cae64
Author: Reynold Xin <rx...@databricks.com>
Date:   2015-02-03T06:16:42Z

    [SPARK-5549] Define TaskContext interface in Scala.
    
    So the interface documentation shows up in ScalaDoc.

----


---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72613931
  
    Going to merge since tests passed previously, and the latest failure was due to a flaky test in streaming.



---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#discussion_r23985360
  
    --- Diff: core/src/main/scala/org/apache/spark/TaskContext.scala ---
    @@ -15,112 +15,116 @@
      * limitations under the License.
      */
     
    -package org.apache.spark;
    +package org.apache.spark
     
    -import java.io.Serializable;
    +import java.io.Serializable
     
    -import scala.Function0;
    -import scala.Function1;
    -import scala.Unit;
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.util.TaskCompletionListener
     
    -import org.apache.spark.annotation.DeveloperApi;
    -import org.apache.spark.executor.TaskMetrics;
    -import org.apache.spark.util.TaskCompletionListener;
     
    -/**
    - * Contextual information about a task which can be read or mutated during
    - * execution. To access the TaskContext for a running task use
    - * TaskContext.get().
    - */
    -public abstract class TaskContext implements Serializable {
    +object TaskContext {
       /**
        * Return the currently active TaskContext. This can be called inside of
        * user functions to access contextual information about running tasks.
        */
    -  public static TaskContext get() {
    -    return taskContext.get();
    -  }
    +  def get(): TaskContext = taskContext.get
    +
    +  private val taskContext: ThreadLocal[TaskContext] = new ThreadLocal[TaskContext]
    +
    +  private[spark] def setTaskContext(tc: TaskContext): Unit = taskContext.set(tc)
    +
    +  private[spark] def unset(): Unit = taskContext.remove()
    +}
     
    -  private static ThreadLocal<TaskContext> taskContext =
    -    new ThreadLocal<TaskContext>();
     
    -  static void setTaskContext(TaskContext tc) {
    -    taskContext.set(tc);
    -  }
    +/**
    + * Contextual information about a task which can be read or mutated during
    + * execution. To access the TaskContext for a running task, use:
    + * {{{
    + *   org.apache.spark.TaskContext.get()
    + * }}}
    + */
    +abstract class TaskContext extends Serializable {
    +  // Note: TaskContext must NOT define a get method. Otherwise it will prevent the Scala compiler
    +  // from generating a static get method (based on the companion object's get method).
     
    -  static void unset() {
    -    taskContext.remove();
    -  }
    +  // Note: getters in this class are defined with parentheses to maintain backward compatibility.
     
       /**
    -   * Whether the task has completed.
    +   * Returns true if the task has completed.
        */
    -  public abstract boolean isCompleted();
    +  def isCompleted(): Boolean
     
       /**
    -   * Whether the task has been killed.
    +   * Returns true if the task has been killed.
        */
    -  public abstract boolean isInterrupted();
    +  def isInterrupted(): Boolean
     
    -  /** @deprecated use {@link #isRunningLocally()} */
    -  @Deprecated
    -  public abstract boolean runningLocally();
    +  /** @deprecated use { @link #isRunningLocally()}*/
    --- End diff --
    
    Scalastyle didn't like the whitespace 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72606843
  
      [Test build #26620 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26620/consoleFull) for   PR 4324 at commit [`2480a17`](https://github.com/apache/spark/commit/2480a177df2023238778fb217fb9b3e4794a9b82).
     * This patch **fails Spark unit 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-5549] Define TaskContext interface in S...

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

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


---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72613282
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26633/
    Test 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72602375
  
      [Test build #26620 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26620/consoleFull) for   PR 4324 at commit [`2480a17`](https://github.com/apache/spark/commit/2480a177df2023238778fb217fb9b3e4794a9b82).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72607656
  
      [Test build #26618 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26618/consoleFull) for   PR 4324 at commit [`573756f`](https://github.com/apache/spark/commit/573756f6c1a7aa3a7bfd490068d8b2d2a459d357).
     * 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72600182
  
      [Test build #26614 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26614/consoleFull) for   PR 4324 at commit [`87dd537`](https://github.com/apache/spark/commit/87dd537e591af257007f86a600e39c97b87cae64).
     * This patch **fails Scala style 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72601740
  
    LGTM; this is a pretty straightforward change.  Compared to implementing this in Java, this gains the benefit of having this class's docs appear in Scaladoc.  I guess there's a potential downside of Java compatibility / usability being broken should we add new methods, but I think we can address that via our review processes.  It might be nice to add a comment at the top of the `TaskContext` and `TaskContextImpl` files referencing the `JavaTaskCompletionListenerImpl` test case so that people remember to update it when adding new methods.


---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72608063
  
      [Test build #26633 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26633/consoleFull) for   PR 4324 at commit [`2480a17`](https://github.com/apache/spark/commit/2480a177df2023238778fb217fb9b3e4794a9b82).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72601943
  
      [Test build #26618 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26618/consoleFull) for   PR 4324 at commit [`573756f`](https://github.com/apache/spark/commit/573756f6c1a7aa3a7bfd490068d8b2d2a459d357).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-5549] Define TaskContext interface in S...

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

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72613272
  
      [Test build #26633 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26633/consoleFull) for   PR 4324 at commit [`2480a17`](https://github.com/apache/spark/commit/2480a177df2023238778fb217fb9b3e4794a9b82).
     * This patch **fails Spark unit 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72601872
  
    I don't know whether MiMa actually checks our Java classes, so that might be another side-benefit of porting this back to Scala.


---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72600186
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26614/
    Test 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#discussion_r23984946
  
    --- Diff: core/src/test/java/test/org/apache/spark/JavaTaskCompletionListenerImpl.java ---
    @@ -15,9 +15,10 @@
      * limitations under the License.
      */
     
    -package org.apache.spark.util;
    +package test.org.apache.spark;
    --- End diff --
    
    note that I changed the package to make sure we test package visibility correctly also. 
    
    We should do this for other Java API tests, but we can do those later.



---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#discussion_r23985377
  
    --- Diff: core/src/main/scala/org/apache/spark/TaskContext.scala ---
    @@ -15,112 +15,116 @@
      * limitations under the License.
      */
     
    -package org.apache.spark;
    +package org.apache.spark
     
    -import java.io.Serializable;
    +import java.io.Serializable
     
    -import scala.Function0;
    -import scala.Function1;
    -import scala.Unit;
    +import org.apache.spark.annotation.DeveloperApi
    +import org.apache.spark.executor.TaskMetrics
    +import org.apache.spark.util.TaskCompletionListener
     
    -import org.apache.spark.annotation.DeveloperApi;
    -import org.apache.spark.executor.TaskMetrics;
    -import org.apache.spark.util.TaskCompletionListener;
     
    -/**
    - * Contextual information about a task which can be read or mutated during
    - * execution. To access the TaskContext for a running task use
    - * TaskContext.get().
    - */
    -public abstract class TaskContext implements Serializable {
    +object TaskContext {
       /**
        * Return the currently active TaskContext. This can be called inside of
        * user functions to access contextual information about running tasks.
        */
    -  public static TaskContext get() {
    -    return taskContext.get();
    -  }
    +  def get(): TaskContext = taskContext.get
    +
    +  private val taskContext: ThreadLocal[TaskContext] = new ThreadLocal[TaskContext]
    +
    +  private[spark] def setTaskContext(tc: TaskContext): Unit = taskContext.set(tc)
    +
    +  private[spark] def unset(): Unit = taskContext.remove()
    +}
     
    -  private static ThreadLocal<TaskContext> taskContext =
    -    new ThreadLocal<TaskContext>();
     
    -  static void setTaskContext(TaskContext tc) {
    -    taskContext.set(tc);
    -  }
    +/**
    + * Contextual information about a task which can be read or mutated during
    + * execution. To access the TaskContext for a running task, use:
    + * {{{
    + *   org.apache.spark.TaskContext.get()
    + * }}}
    + */
    +abstract class TaskContext extends Serializable {
    +  // Note: TaskContext must NOT define a get method. Otherwise it will prevent the Scala compiler
    +  // from generating a static get method (based on the companion object's get method).
     
    -  static void unset() {
    -    taskContext.remove();
    -  }
    +  // Note: getters in this class are defined with parentheses to maintain backward compatibility.
     
       /**
    -   * Whether the task has completed.
    +   * Returns true if the task has completed.
        */
    -  public abstract boolean isCompleted();
    +  def isCompleted(): Boolean
     
       /**
    -   * Whether the task has been killed.
    +   * Returns true if the task has been killed.
        */
    -  public abstract boolean isInterrupted();
    +  def isInterrupted(): Boolean
     
    -  /** @deprecated use {@link #isRunningLocally()} */
    -  @Deprecated
    -  public abstract boolean runningLocally();
    +  /** @deprecated use { @link #isRunningLocally()}*/
    +  @deprecated("1.2.0", "use isRunningLocally")
    +  def runningLocally(): Boolean
     
    -  public abstract boolean isRunningLocally();
    +  /**
    +   * Returns true if the task is running locally in the driver program.
    +   * @return
    --- End diff --
    
    Can probably drop this `@return` tag.


---
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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72606849
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/26620/
    Test 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-5549] Define TaskContext interface in S...

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

    https://github.com/apache/spark/pull/4324#issuecomment-72600044
  
      [Test build #26614 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/26614/consoleFull) for   PR 4324 at commit [`87dd537`](https://github.com/apache/spark/commit/87dd537e591af257007f86a600e39c97b87cae64).
     * This patch merges cleanly.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at 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-5549] Define TaskContext interface in S...

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

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