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

[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

GitHub user mengxr opened a pull request:

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

    [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

    ## What changes were proposed in this pull request?
    
    I made one pass over barrier APIs added to Spark 2.4 and updates some scopes and docs.
    
    TODOs:
    - [ ] scala doc
    - [ ] python doc

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

    $ git pull https://github.com/mengxr/spark SPARK-25248

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

    https://github.com/apache/spark/pull/22240.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 #22240
    
----
commit 19e380ab4f7242f2f2ef48aca81445b0adf0a87d
Author: Xiangrui Meng <me...@...>
Date:   2018-08-27T04:18:54Z

    update barrier Scala doc

----


---

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


[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r212863507
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -21,25 +21,31 @@ import java.util.{Properties, Timer, TimerTask}
     
     import scala.concurrent.duration._
     import scala.language.postfixOps
    -
    -import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since}
     import org.apache.spark.executor.TaskMetrics
     import org.apache.spark.memory.TaskMemoryManager
     import org.apache.spark.metrics.MetricsSystem
     import org.apache.spark.rpc.{RpcEndpointRef, RpcTimeout}
     import org.apache.spark.util.{RpcUtils, Utils}
     
    -/** A [[TaskContext]] with extra info and tooling for a barrier stage. */
    -class BarrierTaskContext(
    +/**
    + * :: Experimental ::
    + * A [[TaskContext]] with extra contextual info and tooling for tasks in a barrier stage.
    + * Use [[BarrierTaskContext#get]] to obtain the barrier context for a running barrier task.
    + */
    +@Experimental
    +@Since("2.4.0")
    +class BarrierTaskContext private[spark] (
    --- End diff --
    
    Made the constructor package private to force users get it from `#get()`.


---

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


[GitHub] spark pull request #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r214229966
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -145,20 +143,77 @@ class BarrierTaskContext(
     
       /**
        * :: Experimental ::
    -   * Returns the all task infos in this barrier stage, the task infos are ordered by partitionId.
    +   * Returns [[BarrierTaskInfo]] for all tasks in this barrier stage, ordered by partition ID.
        */
       @Experimental
       @Since("2.4.0")
       def getTaskInfos(): Array[BarrierTaskInfo] = {
         val addressesStr = localProperties.getProperty("addresses", "")
         addressesStr.split(",").map(_.trim()).map(new BarrierTaskInfo(_))
       }
    +
    +  // delegate methods
    +
    +  override def isCompleted(): Boolean = taskContext.isCompleted()
    +
    +  override def isInterrupted(): Boolean = taskContext.isInterrupted()
    +
    +  override def isRunningLocally(): Boolean = taskContext.isRunningLocally()
    +
    +  override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = {
    +    taskContext.addTaskCompletionListener(listener)
    +    this
    +  }
    +
    +  override def addTaskFailureListener(listener: TaskFailureListener): this.type = {
    +    taskContext.addTaskFailureListener(listener)
    +    this
    +  }
    +
    +  override def stageId(): Int = taskContext.stageId()
    +
    +  override def stageAttemptNumber(): Int = taskContext.stageAttemptNumber()
    +
    +  override def partitionId(): Int = taskContext.partitionId()
    +
    +  override def attemptNumber(): Int = taskContext.attemptNumber()
    +
    +  override def taskAttemptId(): Long = taskContext.taskAttemptId()
    +
    +  override def getLocalProperty(key: String): String = taskContext.getLocalProperty(key)
    +
    +  override def taskMetrics(): TaskMetrics = taskContext.taskMetrics()
    +
    +  override def getMetricsSources(sourceName: String): Seq[Source] = {
    +    taskContext.getMetricsSources(sourceName)
    +  }
    +
    +  override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted()
    +
    +  override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason()
    +
    +  override private[spark] def taskMemoryManager(): TaskMemoryManager = {
    +    taskContext.taskMemoryManager()
    +  }
    +
    +  override private[spark] def registerAccumulator(a: AccumulatorV2[_, _]): Unit = {
    +    taskContext.registerAccumulator(a)
    +  }
    +
    +  override private[spark] def setFetchFailed(fetchFailed: FetchFailedException): Unit = {
    +    taskContext.setFetchFailed(fetchFailed)
    +  }
     }
     
    +@Experimental
    +@Since("2.4.0")
     object BarrierTaskContext {
       /**
    +   * :: Experimental ::
        * Return the currently active BarrierTaskContext. This can be called inside of user functions to
    --- End diff --
    
    nit: `Return` -> `Returns`?


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95639 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95639/testReport)** for PR 22240 at commit [`b6a3c5b`](https://github.com/apache/spark/commit/b6a3c5b3de3ef145805542511770da4f59886858).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r212863444
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -68,7 +74,7 @@ class BarrierTaskContext(
        *
        * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all
        * possible code branches. Otherwise, you may get the job hanging or a SparkException after
    -   * timeout. Some examples of misuses listed below:
    +   * timeout. Some examples of '''misuses''' listed below:
    --- End diff --
    
    use bold font to make sure users don't misread


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2807/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2814/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2627/
    Test PASSed.


---

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


[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r212863381
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -21,25 +21,31 @@ import java.util.{Properties, Timer, TimerTask}
     
     import scala.concurrent.duration._
     import scala.language.postfixOps
    -
    -import org.apache.spark.annotation.{Experimental, Since}
    +import org.apache.spark.annotation.{DeveloperApi, Experimental, Since}
     import org.apache.spark.executor.TaskMetrics
     import org.apache.spark.memory.TaskMemoryManager
     import org.apache.spark.metrics.MetricsSystem
     import org.apache.spark.rpc.{RpcEndpointRef, RpcTimeout}
     import org.apache.spark.util.{RpcUtils, Utils}
     
    -/** A [[TaskContext]] with extra info and tooling for a barrier stage. */
    -class BarrierTaskContext(
    +/**
    + * :: Experimental ::
    + * A [[TaskContext]] with extra contextual info and tooling for tasks in a barrier stage.
    + * Use [[BarrierTaskContext#get]] to obtain the barrier context for a running barrier task.
    + */
    +@Experimental
    +@Since("2.4.0")
    +class BarrierTaskContext private[spark] (
         override val stageId: Int,
         override val stageAttemptNumber: Int,
         override val partitionId: Int,
         override val taskAttemptId: Long,
         override val attemptNumber: Int,
    -    override val taskMemoryManager: TaskMemoryManager,
    +    private[spark] override val taskMemoryManager: TaskMemoryManager,
    --- End diff --
    
    This is not exposed by `TaskContext`.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2809/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95418/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2828/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95280 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95280/testReport)** for PR 22240 at commit [`c2624ed`](https://github.com/apache/spark/commit/c2624eda86defd272be8274d56c4a759fb4efb45).
     * This patch **fails to build**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95639/
    Test FAILed.


---

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


[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r212863571
  
    --- Diff: core/src/main/scala/org/apache/spark/rdd/RDDBarrier.scala ---
    @@ -22,15 +22,22 @@ import scala.reflect.ClassTag
     import org.apache.spark.TaskContext
     import org.apache.spark.annotation.{Experimental, Since}
     
    -/** Represents an RDD barrier, which forces Spark to launch tasks of this stage together. */
    -class RDDBarrier[T: ClassTag](rdd: RDD[T]) {
    +/**
    + * :: Experimental ::
    + * Wraps an RDD in a barrier stage, which forces Spark to launch tasks of this stage together.
    + * [[RDDBarrier]] instances are created by [[RDD.barrier]].
    + */
    +@Experimental
    +@Since("2.4.0")
    +class RDDBarrier[T: ClassTag] private[spark] (rdd: RDD[T]) {
    --- End diff --
    
    also hide the constructor here


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95640 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95640/testReport)** for PR 22240 at commit [`ae4a8e6`](https://github.com/apache/spark/commit/ae4a8e6b784519a2f2a237be258ed1059e91be64).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r212863543
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskInfo.scala ---
    @@ -28,4 +28,4 @@ import org.apache.spark.annotation.{Experimental, Since}
      */
     @Experimental
     @Since("2.4.0")
    -class BarrierTaskInfo(val address: String)
    +class BarrierTaskInfo private[spark] (val address: String)
    --- End diff --
    
    hide the constructor since this is not to be constructed by user


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95280/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95641/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95379 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95379/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95640/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95379 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95379/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95640 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95640/testReport)** for PR 22240 at commit [`ae4a8e6`](https://github.com/apache/spark/commit/ae4a8e6b784519a2f2a237be258ed1059e91be64).


---

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


[GitHub] spark pull request #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r213537490
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -68,7 +74,7 @@ class BarrierTaskContext(
        *
        * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all
        * possible code branches. Otherwise, you may get the job hanging or a SparkException after
    -   * timeout. Some examples of misuses listed below:
    +   * timeout. Some examples of '''misuses''' listed below:
    --- End diff --
    
    just saw it, will include it if Jenkins fails:)


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95398 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95398/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95361 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95361/testReport)** for PR 22240 at commit [`7e2975b`](https://github.com/apache/spark/commit/7e2975bd4f723869cded56dc1269e00d7e03df1f).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95642/testReport)** for PR 22240 at commit [`c61eec3`](https://github.com/apache/spark/commit/c61eec363f78d586070c673e44e9120eb10b83b5).


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2566/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    LGTM


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2656/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2670/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

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


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    retest this please


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95276 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95276/testReport)** for PR 22240 at commit [`19e380a`](https://github.com/apache/spark/commit/19e380ab4f7242f2f2ef48aca81445b0adf0a87d).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    test this please


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95379/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95648/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95329 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95329/testReport)** for PR 22240 at commit [`7e2975b`](https://github.com/apache/spark/commit/7e2975bd4f723869cded56dc1269e00d7e03df1f).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95648 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95648/testReport)** for PR 22240 at commit [`9b6a47b`](https://github.com/apache/spark/commit/9b6a47bf718309eb0b5a22a0282a5a7c4226e991).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by mengxr <gi...@git.apache.org>.
Github user mengxr commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    Merged into master. Thanks for review!


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2602/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2641/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95661 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95661/testReport)** for PR 22240 at commit [`9b6a47b`](https://github.com/apache/spark/commit/9b6a47bf718309eb0b5a22a0282a5a7c4226e991).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95418 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95418/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r213377591
  
    --- Diff: core/src/main/scala/org/apache/spark/BarrierTaskContext.scala ---
    @@ -68,7 +74,7 @@ class BarrierTaskContext(
        *
        * CAUTION! In a barrier stage, each task must have the same number of barrier() calls, in all
        * possible code branches. Otherwise, you may get the job hanging or a SparkException after
    -   * timeout. Some examples of misuses listed below:
    +   * timeout. Some examples of '''misuses''' listed below:
    --- End diff --
    
    nit: `listed` -> `are listed`?


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95648 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95648/testReport)** for PR 22240 at commit [`9b6a47b`](https://github.com/apache/spark/commit/9b6a47bf718309eb0b5a22a0282a5a7c4226e991).
     * This patch **fails due to an unknown error code, -9**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark pull request #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for...

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

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


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95280 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95280/testReport)** for PR 22240 at commit [`c2624ed`](https://github.com/apache/spark/commit/c2624eda86defd272be8274d56c4a759fb4efb45).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95661 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95661/testReport)** for PR 22240 at commit [`9b6a47b`](https://github.com/apache/spark/commit/9b6a47bf718309eb0b5a22a0282a5a7c4226e991).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95276/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95329 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95329/testReport)** for PR 22240 at commit [`7e2975b`](https://github.com/apache/spark/commit/7e2975bd4f723869cded56dc1269e00d7e03df1f).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95639 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95639/testReport)** for PR 22240 at commit [`b6a3c5b`](https://github.com/apache/spark/commit/b6a3c5b3de3ef145805542511770da4f59886858).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95418 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95418/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95642/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95361 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95361/testReport)** for PR 22240 at commit [`7e2975b`](https://github.com/apache/spark/commit/7e2975bd4f723869cded56dc1269e00d7e03df1f).
     * This patch **fails to generate documentation**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2806/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2563/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95642/testReport)** for PR 22240 at commit [`c61eec3`](https://github.com/apache/spark/commit/c61eec363f78d586070c673e44e9120eb10b83b5).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95641 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95641/testReport)** for PR 22240 at commit [`47ebd08`](https://github.com/apache/spark/commit/47ebd0849ec3344f05eb8eb74df36d7bfda7e130).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95398/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark pull request #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for...

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

    https://github.com/apache/spark/pull/22240#discussion_r213754911
  
    --- Diff: core/src/main/scala/org/apache/spark/scheduler/Task.scala ---
    @@ -82,31 +82,22 @@ private[spark] abstract class Task[T](
         SparkEnv.get.blockManager.registerTask(taskAttemptId)
         // TODO SPARK-24874 Allow create BarrierTaskContext based on partitions, instead of whether
         // the stage is barrier.
    -    context = if (isBarrier) {
    --- End diff --
    
    `context` is still used in the following statements.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/2808/
    Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95398 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95398/testReport)** for PR 22240 at commit [`365e7b8`](https://github.com/apache/spark/commit/365e7b8c0dc161b765476cffb59c0a174d6f85ae).


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by jiangxb1987 <gi...@git.apache.org>.
Github user jiangxb1987 commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    retest this please


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test PASSed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Merged build finished. Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95329/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [WIP] [SPARK-25248] [CORE] Audit barrier APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95276 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95276/testReport)** for PR 22240 at commit [`19e380a`](https://github.com/apache/spark/commit/19e380ab4f7242f2f2ef48aca81445b0adf0a87d).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/95361/
    Test FAILed.


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

Posted by kiszk <gi...@git.apache.org>.
Github user kiszk commented on the issue:

    https://github.com/apache/spark/pull/22240
  
    retest this please


---

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


[GitHub] spark issue #22240: [SPARK-25248] [CORE] Audit barrier Scala APIs for 2.4

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

    https://github.com/apache/spark/pull/22240
  
    **[Test build #95641 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/95641/testReport)** for PR 22240 at commit [`47ebd08`](https://github.com/apache/spark/commit/47ebd0849ec3344f05eb8eb74df36d7bfda7e130).
     * This patch **fails MiMa tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

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