You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by yhuai <gi...@git.apache.org> on 2015/10/26 06:58:27 UTC

[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

GitHub user yhuai opened a pull request:

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

    [SPARK-9858][SPARK-9859][SPARK-9861][SQL][WIP] Add an ExchangeCoordinator to estimate the number of post-shuffle partitions for aggregates and joins

    https://issues.apache.org/jira/browse/SPARK-9858
    https://issues.apache.org/jira/browse/SPARK-9859
    https://issues.apache.org/jira/browse/SPARK-9861
    
    Will fill in more details. Send out the PR first to let jenkins test it.

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

    $ git pull https://github.com/yhuai/spark numReducer

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

    https://github.com/apache/spark/pull/9276.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 #9276
    
----
commit 18740c3c7241e58dfe655d90ef12bc088ab5b4f1
Author: Yin Huai <yh...@databricks.com>
Date:   2015-10-21T05:45:18Z

    Make ShuffledRowRDD support fetching multiple parent partitions.

commit 6fd0594a913d9a969caa49aaf798ca9d718be44e
Author: Yin Huai <yh...@databricks.com>
Date:   2015-10-23T05:15:48Z

    Add an ExchangeCoordinator to estimate the number of post-shuffle partitions.

commit d392656ad8cde9230320203ab360030ded4616ed
Author: Yin Huai <yh...@databricks.com>
Date:   2015-10-26T05:55:03Z

    Change settings for testing purpose.

----


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152352368
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153240695
  
    **[Test build #44879 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44879/consoleFull)** for PR 9276 at commit [`4a99729`](https://github.com/apache/spark/commit/4a997295a3e68ba9b137e9f65fac89c2cf4a0cfc).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153147492
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44818/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152916977
  
    **[Test build #44792 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44792/consoleFull)** for PR 9276 at commit [`36b6dbc`](https://github.com/apache/spark/commit/36b6dbc042b5a127ba230b6bf425259ae7e5a7ac).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43333545
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -129,8 +143,29 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
         }
       }
     
    -  protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") {
    +  override protected def doPrepare(): Unit = {
    +    // If an ExchangeCoordinator is needed, we register this Exchange operator
    +    // to the coordinator when we do prepare. It is important to make sure
    +    // we register this operator right before the execution instead of register it
    +    // in the constructor.
    --- End diff --
    
    Can you comment why that is required briefly?


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151719680
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44487/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152309820
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r43059732
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala ---
    @@ -233,6 +233,25 @@ private[spark] object SQLConf {
         defaultValue = Some(200),
         doc = "The default number of partitions to use when shuffling data for joins or aggregations.")
     
    +  val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE =
    +    longConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize",
    +      defaultValue = Some(64 * 1024 * 1024),
    +      doc = "The target post-shuffle input size of a task.")
    --- End diff --
    
    ", in bytes" ?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153282638
  
    **[Test build #44882 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44882/consoleFull)** for PR 9276 at commit [`60e371e`](https://github.com/apache/spark/commit/60e371e7d0d973745ed40fa3870e9446490224de).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152877325
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152324779
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44632/
    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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151671903
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719723
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -181,7 +215,54 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
             }
           }
         }
    -    new ShuffledRowRDD(rddWithPartitionIds, serializer, part.numPartitions)
    +
    +    // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds
    +    // are in the form of (partitionId, row) and every partitionId is in the expected range
    +    // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough.
    +    val dependency =
    +      new ShuffleDependency[Int, InternalRow, InternalRow](
    +        rddWithPartitionIds,
    +        new PartitionIdPassthrough(part.numPartitions),
    +        Some(serializer))
    +
    +    dependency
    +  }
    +
    +  /**
    +   * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset.
    +   * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional
    +   * partition start indices array. If this optional array is defined, the returned
    +   * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array.
    +   */
    +  private[sql] def preparePostShuffleRDD(
    +      shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow],
    +      specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = {
    +    // If an array of partition start indices is provided, we need to use this array
    +    // to create the ShuffledRowRDD. Also, we need to update newPartitioning to
    +    // update the number of post-shuffle partitions.
    +    specifiedPartitionStartIndices.foreach { indices =>
    +      assert(newPartitioning.isInstanceOf[HashPartitioning])
    +      newPartitioning = newPartitioning.withNumPartitions(indices.length)
    --- End diff --
    
    Why not create a new `UnknownPartitioning` with the specified number of partitions? Based on your comments later in this file, it seems like it's not really correct to compare a `HashPartitioning(k)`s that was created via coalescing to one that was non-coalesced, so using `UnknownPartitioning` might help to avoid this. Just wondering if there are any tricky correctness corner-cases that we might hit as a result of this line. 


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153240788
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44879/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153251495
  
    **[Test build #44882 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44882/consoleFull)** for PR 9276 at commit [`60e371e`](https://github.com/apache/spark/commit/60e371e7d0d973745ed40fa3870e9446490224de).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43334111
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -202,12 +286,103 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
       private def canonicalPartitioning(requiredDistribution: Distribution): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    -      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions)
    +      case ClusteredDistribution(clustering) =>
    +        HashPartitioning(clustering, numPreShufflePartitions)
    +      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPreShufflePartitions)
           case dist => sys.error(s"Do not know how to satisfy distribution $dist")
         }
       }
     
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val needsCoordinator =
    --- End diff --
    
    needsCoordinator -> supportsCoordinator is more apt yes?


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42962062
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala ---
    @@ -0,0 +1,466 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.test.TestSQLContext
    +import org.apache.spark.sql._
    +import org.apache.spark.{SparkFunSuite, SparkContext, SparkConf, MapOutputStatistics}
    +
    +class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll {
    --- End diff --
    
    Probably there are some out-dated comments. I will clean this suite up.


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

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


[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151719678
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720595
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -181,7 +215,54 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
             }
           }
         }
    -    new ShuffledRowRDD(rddWithPartitionIds, serializer, part.numPartitions)
    +
    +    // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds
    +    // are in the form of (partitionId, row) and every partitionId is in the expected range
    +    // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough.
    +    val dependency =
    +      new ShuffleDependency[Int, InternalRow, InternalRow](
    +        rddWithPartitionIds,
    +        new PartitionIdPassthrough(part.numPartitions),
    +        Some(serializer))
    +
    +    dependency
    +  }
    +
    +  /**
    +   * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset.
    +   * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional
    +   * partition start indices array. If this optional array is defined, the returned
    +   * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array.
    +   */
    +  private[sql] def preparePostShuffleRDD(
    +      shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow],
    +      specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = {
    +    // If an array of partition start indices is provided, we need to use this array
    +    // to create the ShuffledRowRDD. Also, we need to update newPartitioning to
    +    // update the number of post-shuffle partitions.
    +    specifiedPartitionStartIndices.foreach { indices =>
    +      assert(newPartitioning.isInstanceOf[HashPartitioning])
    +      newPartitioning = newPartitioning.withNumPartitions(indices.length)
    --- End diff --
    
    This change makes sense.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152785672
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44745/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152877559
  
    **[Test build #44781 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44781/consoleFull)** for PR 9276 at commit [`890521e`](https://github.com/apache/spark/commit/890521e54d81f8ef8172a43fff50f824a7fec1be).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153237165
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151671858
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151037502
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44340/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43473950
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean indicates if this coordinator has made decision on how to shuffle data.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // At here, we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage if the number of partitions of the RDD is 0.
    +    require(mapOutputStatistics.length <= numExchanges)
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    require(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    +        "among registered Exchange operator.")
    +
    +    val numPreShufflePartitions = distinctNumPreShufflePartitions.head
    +    val partitionStartIndices = ArrayBuffer[Int]()
    +    var postShuffleInputSize = 0L
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // The first element of partitionStartIndices is always 0.
    +    partitionStartIndices += 0
    +
    +    var i = 0
    +    while (i < numPreShufflePartitions) {
    +      // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages.
    +      // Then, we add the total size to postShuffleInputSize.
    +      var j = 0
    +      while (j < mapOutputStatistics.length) {
    +        postShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i)
    +        j += 1
    +      }
    +
    +      // If the current postShuffleInputSize is equal or greater than the
    +      // targetPostShuffleInputSize, We need to add a new element in partitionStartIndices.
    +      if (postShuffleInputSize >= targetPostShuffleInputSize) {
    +        if (i < numPreShufflePartitions - 1) {
    +          // Next start index.
    +          partitionStartIndices += i + 1
    +        } else {
    +          // This is the last element. So, we do not need to append the next start index to
    +          // partitionStartIndices.
    +        }
    +        // reset postShuffleInputSize.
    +        postShuffleInputSize = 0L
    +      }
    +
    +      i += 1
    +    }
    +
    +    partitionStartIndices.toArray
    +  }
    +
    +  private def doEstimationIfNecessary(): Unit = {
    +    if (!estimated) {
    +      // Make sure we have the expected number of registered Exchange operators.
    +      require(exchanges.length == numExchanges)
    +
    +      val newPostShuffleRDDs = new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +      // Submit all map stages
    +      val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]()
    +      val submittedStageFutures = ArrayBuffer[SimpleFutureAction[MapOutputStatistics]]()
    +      var i = 0
    +      while (i < numExchanges) {
    +        val exchange = exchanges(i)
    +        val shuffleDependency = exchange.prepareShuffleDependency()
    +        shuffleDependencies += shuffleDependency
    +        if (shuffleDependency.rdd.partitions.length != 0) {
    +          // submitMapStage does not accept RDD with 0 partition.
    +          // So, we will not submit this dependency.
    +          submittedStageFutures +=
    +            exchange.sqlContext.sparkContext.submitMapStage(shuffleDependency)
    +        }
    +        i += 1
    +      }
    +
    +      // Wait for the finishes of those submitted map stages.
    +      val mapOutputStatistics = new Array[MapOutputStatistics](submittedStageFutures.length)
    +      i = 0
    +      while (i < submittedStageFutures.length) {
    +        // This call is a blocking call. If the stage has not finished, we will wait at here.
    +        mapOutputStatistics(i) = submittedStageFutures(i).get()
    +        i += 1
    +      }
    +
    +      // Now, we estimate partitionStartIndices. partitionStartIndices.length will be the
    +      // number of post-shuffle partitions.
    +      val partitionStartIndices =
    +        if (mapOutputStatistics.length == 0) {
    +          None
    +        } else {
    +          Some(estimatePartitionStartIndices(mapOutputStatistics))
    +        }
    +
    +      i = 0
    +      while (i < numExchanges) {
    +        val exchange = exchanges(i)
    +        val rdd =
    +          exchange.preparePostShuffleRDD(shuffleDependencies(i), partitionStartIndices)
    +        newPostShuffleRDDs.put(exchange, rdd)
    +
    +        i += 1
    +      }
    +
    +      // Finally, we acquire a lock and set newPostShuffleRDDs and estimated.
    +      this.synchronized {
    --- End diff --
    
    If this is called from multiple threads, how does the loop on line 188-198 work? Why wait here instead of before any of the work happens, say line 178


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42961903
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -465,6 +465,6 @@ private[hive] object TestHiveContext {
       val overrideConfs: Map[String, String] =
         Map(
           // Fewer shuffle partitions to speed up testing.
    -      SQLConf.SHUFFLE_PARTITIONS.key -> "5"
    +      SQLConf.SHUFFLE_PARTITIONS.key -> "10"
    --- End diff --
    
    This change is for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151041899
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719127
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -209,6 +299,98 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
         }
       }
     
    +  /**
    +   * Adds [[ExchangeCoordinator]] to [[Exchange]]s if adaptive query execution is enabled
    +   * and partitioning schemes of these [[Exchange]]s support [[ExchangeCoordinator]].
    +   */
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val supportsCoordinator =
    +      if (children.exists(_.isInstanceOf[Exchange])) {
    +        // Right now, ExchangeCoordinator only support HashPartitionings.
    +        children.forall {
    +          case e @ Exchange(hash: HashPartitioning, _, _) => true
    +          case child =>
    +            child.outputPartitioning match {
    +              case hash: HashPartitioning => true
    +              case collection: PartitioningCollection =>
    +                collection.partitionings.exists(_.isInstanceOf[HashPartitioning])
    --- End diff --
    
    Is it possible for a `PartitioningCollection` to contain partitioning of different classes (e.g. a hash partitioning _and_ a range partitioning)? I don't think so, so maybe it'd be clearer to use `forall` instead of `exists`?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720073
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    Why should this be true? What if I'm joining together two relations, one of which originally has 200 map partitions and one of which has 100? The resulting exchanges must produce the same number of partitions, but their inputs don't have to, right? Am I overlooking something obvious?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152016561
  
    **[Test build #44544 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44544/consoleFull)** for PR 9276 at commit [`24e1caf`](https://github.com/apache/spark/commit/24e1cafba9479fc5101ef16be615fc941fce3a21).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152328867
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42961923
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala ---
    @@ -233,6 +233,25 @@ private[spark] object SQLConf {
         defaultValue = Some(200),
         doc = "The default number of partitions to use when shuffling data for joins or aggregations.")
     
    +  val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE =
    +    longConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize",
    +      defaultValue = Some(64 * 1024 * 1024),
    +      doc = "The target post-shuffle input size of a task.")
    +
    +  val ADAPTIVE_EXECUTION_ENABLED = booleanConf("spark.sql.adaptive.enabled",
    +    defaultValue = Some(true),
    --- End diff --
    
    We may disable it by default.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151694676
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44472/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152303848
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152308560
  
    test this please


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

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


[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152877321
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42961911
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala ---
    @@ -233,6 +233,25 @@ private[spark] object SQLConf {
         defaultValue = Some(200),
         doc = "The default number of partitions to use when shuffling data for joins or aggregations.")
     
    +  val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE =
    +    longConf("spark.sql.adaptive.shuffle.targetPostShuffleInputSize",
    +      defaultValue = Some(64 * 1024 * 1024),
    +      doc = "The target post-shuffle input size of a task.")
    +
    +  val ADAPTIVE_EXECUTION_ENABLED = booleanConf("spark.sql.adaptive.enabled",
    +    defaultValue = Some(true),
    +    doc = "When true, enable adaptive query execution.")
    +
    +  val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS =
    +    intConf("spark.sql.adaptive.minNumPostShufflePartitions",
    +      defaultValue = Some(4),
    --- End diff --
    
    This change is for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719283
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -209,6 +299,98 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
         }
       }
     
    +  /**
    +   * Adds [[ExchangeCoordinator]] to [[Exchange]]s if adaptive query execution is enabled
    +   * and partitioning schemes of these [[Exchange]]s support [[ExchangeCoordinator]].
    +   */
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val supportsCoordinator =
    +      if (children.exists(_.isInstanceOf[Exchange])) {
    +        // Right now, ExchangeCoordinator only support HashPartitionings.
    +        children.forall {
    +          case e @ Exchange(hash: HashPartitioning, _, _) => true
    +          case child =>
    +            child.outputPartitioning match {
    +              case hash: HashPartitioning => true
    +              case collection: PartitioningCollection =>
    +                collection.partitionings.exists(_.isInstanceOf[HashPartitioning])
    --- End diff --
    
    That will not happen. I will change it to `forall`.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151694513
  
    **[Test build #44472 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44472/consoleFull)** for PR 9276 at commit [`e31ed6b`](https://github.com/apache/spark/commit/e31ed6b1ab6349a0357d2343d101eba9ea47078b).
     * 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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151672519
  
    **[Test build #44472 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44472/consoleFull)** for PR 9276 at commit [`e31ed6b`](https://github.com/apache/spark/commit/e31ed6b1ab6349a0357d2343d101eba9ea47078b).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720524
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    Their inputs RDD do not need to have the same number of partitions. But, once we create the shuffle dependency, they will have the same number of pre-shuffle partitions.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152072524
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151674267
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44473/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152785269
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152007218
  
    https://github.com/yhuai/spark/commit/24e1cafba9479fc5101ef16be615fc941fce3a21 is only for testing purpose. We will remove it before we commit the code.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152303820
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153282983
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44882/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720838
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    Ah, so that's where I think the confusion might be: here we're talking about coalesced partitions that are formed from continuous ranges of map output, whereas I was using "partitions" to refer to the individual map outputs.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577950
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean indicates if this coordinator has made decision on how to shuffle data.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // At here, we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    --- End diff --
    
    Done


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153147487
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152940850
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719732
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -181,7 +215,54 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
             }
           }
         }
    -    new ShuffledRowRDD(rddWithPartitionIds, serializer, part.numPartitions)
    +
    +    // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds
    +    // are in the form of (partitionId, row) and every partitionId is in the expected range
    +    // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough.
    +    val dependency =
    +      new ShuffleDependency[Int, InternalRow, InternalRow](
    +        rddWithPartitionIds,
    +        new PartitionIdPassthrough(part.numPartitions),
    +        Some(serializer))
    +
    +    dependency
    +  }
    +
    +  /**
    +   * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset.
    +   * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional
    +   * partition start indices array. If this optional array is defined, the returned
    +   * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array.
    +   */
    +  private[sql] def preparePostShuffleRDD(
    +      shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow],
    +      specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = {
    +    // If an array of partition start indices is provided, we need to use this array
    +    // to create the ShuffledRowRDD. Also, we need to update newPartitioning to
    +    // update the number of post-shuffle partitions.
    +    specifiedPartitionStartIndices.foreach { indices =>
    +      assert(newPartitioning.isInstanceOf[HashPartitioning])
    +      newPartitioning = newPartitioning.withNumPartitions(indices.length)
    +    }
    +    new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices)
    +  }
    +
    +  protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") {
    +    coordinator match {
    +      case Some(exchangeCoordinator) =>
    +        val shuffleRDD = exchangeCoordinator.postShuffleRDD(this)
    +        assert(shuffleRDD.partitions.length == newPartitioning.numPartitions)
    +        shuffleRDD
    +      case None =>
    +        val shuffleDependency = prepareShuffleDependency()
    +        preparePostShuffleRDD(shuffleDependency)
    +    }
    +  }
    +}
    +
    +object Exchange {
    +  def apply(newPartitioning: Partitioning, child: SparkPlan): Exchange = {
    +    Exchange(newPartitioning, child, None: Option[ExchangeCoordinator])
    --- End diff --
    
    `exchangeCoordinator = None`?


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

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

    https://github.com/apache/spark/pull/9276#discussion_r43722690
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    +        "among registered Exchange operator.")
    +    val numPreShufflePartitions = distinctNumPreShufflePartitions.head
    +
    +    val partitionStartIndices = ArrayBuffer[Int]()
    +    // The first element of partitionStartIndices is always 0.
    +    partitionStartIndices += 0
    +
    +    var postShuffleInputSize = 0L
    +
    +    var i = 0
    +    while (i < numPreShufflePartitions) {
    +      // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages.
    +      // Then, we add the total size to postShuffleInputSize.
    +      var j = 0
    +      while (j < mapOutputStatistics.length) {
    +        postShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i)
    +        j += 1
    +      }
    +
    +      // If the current postShuffleInputSize is equal or greater than the
    +      // targetPostShuffleInputSize, We need to add a new element in partitionStartIndices.
    +      if (postShuffleInputSize >= targetPostShuffleInputSize) {
    +        if (i < numPreShufflePartitions - 1) {
    +          // Next start index.
    +          partitionStartIndices += i + 1
    +        } else {
    +          // This is the last element. So, we do not need to append the next start index to
    +          // partitionStartIndices.
    +        }
    +        // reset postShuffleInputSize.
    +        postShuffleInputSize = 0L
    +      }
    +
    +      i += 1
    +    }
    +
    +    partitionStartIndices.toArray
    +  }
    +
    +  private def doEstimationIfNecessary(): Unit = synchronized {
    +    // It is unlikely that this method will be called from multiple threads
    +    // (when multiple threads trigger the execution of THIS physical)
    +    // because in common use cases, we will create new physical plan after
    +    // users apply operations (e.g. projection) to an existing DataFrame.
    +    // However, if it happens, we have synchronized to make sure only one
    +    // thread will trigger the job submission.
    +    if (!estimated) {
    +      // Make sure we have the expected number of registered Exchange operators.
    +      assert(exchanges.length == numExchanges)
    +
    +      val newPostShuffleRDDs = new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +      // Submit all map stages
    +      val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]()
    +      val submittedStageFutures = ArrayBuffer[SimpleFutureAction[MapOutputStatistics]]()
    +      var i = 0
    +      while (i < numExchanges) {
    +        val exchange = exchanges(i)
    +        val shuffleDependency = exchange.prepareShuffleDependency()
    +        shuffleDependencies += shuffleDependency
    +        if (shuffleDependency.rdd.partitions.length != 0) {
    +          // submitMapStage does not accept RDD with 0 partition.
    +          // So, we will not submit this dependency.
    +          submittedStageFutures +=
    +            exchange.sqlContext.sparkContext.submitMapStage(shuffleDependency)
    +        }
    +        i += 1
    +      }
    +
    +      // Wait for the finishes of those submitted map stages.
    +      val mapOutputStatistics = new Array[MapOutputStatistics](submittedStageFutures.length)
    +      i = 0
    --- End diff --
    
    could use separate variables rather than re-initializing.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152011906
  
    **[Test build #44544 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44544/consoleFull)** for PR 9276 at commit [`24e1caf`](https://github.com/apache/spark/commit/24e1cafba9479fc5101ef16be615fc941fce3a21).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153240784
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152785261
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43723253
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    I will add a comment to emphasize that we rely on the static setting of `spark.sql.shuffle.partitions` to make sure we only have a single distinct value of the number of map output partitions.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720999
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    I see. `distinctNumPreShufflePartitions` at here is the number of map output partitions. For example, if we use hash partitioning with 100 partitions, this value will be 100. Then, after the estimation, we may end up with 10 coalesced partitions.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577893
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean indicates if this coordinator has made decision on how to shuffle data.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // At here, we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage if the number of partitions of the RDD is 0.
    +    require(mapOutputStatistics.length <= numExchanges)
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    require(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    +        "among registered Exchange operator.")
    +
    +    val numPreShufflePartitions = distinctNumPreShufflePartitions.head
    +    val partitionStartIndices = ArrayBuffer[Int]()
    +    var postShuffleInputSize = 0L
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // The first element of partitionStartIndices is always 0.
    +    partitionStartIndices += 0
    +
    +    var i = 0
    +    while (i < numPreShufflePartitions) {
    +      // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages.
    +      // Then, we add the total size to postShuffleInputSize.
    +      var j = 0
    +      while (j < mapOutputStatistics.length) {
    +        postShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i)
    +        j += 1
    +      }
    +
    +      // If the current postShuffleInputSize is equal or greater than the
    +      // targetPostShuffleInputSize, We need to add a new element in partitionStartIndices.
    +      if (postShuffleInputSize >= targetPostShuffleInputSize) {
    +        if (i < numPreShufflePartitions - 1) {
    +          // Next start index.
    +          partitionStartIndices += i + 1
    +        } else {
    +          // This is the last element. So, we do not need to append the next start index to
    +          // partitionStartIndices.
    +        }
    +        // reset postShuffleInputSize.
    +        postShuffleInputSize = 0L
    +      }
    +
    +      i += 1
    +    }
    +
    +    partitionStartIndices.toArray
    +  }
    +
    +  private def doEstimationIfNecessary(): Unit = {
    +    if (!estimated) {
    +      // Make sure we have the expected number of registered Exchange operators.
    +      require(exchanges.length == numExchanges)
    +
    +      val newPostShuffleRDDs = new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +      // Submit all map stages
    +      val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]()
    +      val submittedStageFutures = ArrayBuffer[SimpleFutureAction[MapOutputStatistics]]()
    +      var i = 0
    +      while (i < numExchanges) {
    +        val exchange = exchanges(i)
    +        val shuffleDependency = exchange.prepareShuffleDependency()
    +        shuffleDependencies += shuffleDependency
    +        if (shuffleDependency.rdd.partitions.length != 0) {
    +          // submitMapStage does not accept RDD with 0 partition.
    +          // So, we will not submit this dependency.
    +          submittedStageFutures +=
    +            exchange.sqlContext.sparkContext.submitMapStage(shuffleDependency)
    +        }
    +        i += 1
    +      }
    +
    +      // Wait for the finishes of those submitted map stages.
    +      val mapOutputStatistics = new Array[MapOutputStatistics](submittedStageFutures.length)
    +      i = 0
    +      while (i < submittedStageFutures.length) {
    +        // This call is a blocking call. If the stage has not finished, we will wait at here.
    +        mapOutputStatistics(i) = submittedStageFutures(i).get()
    +        i += 1
    +      }
    +
    +      // Now, we estimate partitionStartIndices. partitionStartIndices.length will be the
    +      // number of post-shuffle partitions.
    +      val partitionStartIndices =
    +        if (mapOutputStatistics.length == 0) {
    +          None
    +        } else {
    +          Some(estimatePartitionStartIndices(mapOutputStatistics))
    +        }
    +
    +      i = 0
    +      while (i < numExchanges) {
    +        val exchange = exchanges(i)
    +        val rdd =
    +          exchange.preparePostShuffleRDD(shuffleDependencies(i), partitionStartIndices)
    +        newPostShuffleRDDs.put(exchange, rdd)
    +
    +        i += 1
    +      }
    +
    +      // Finally, we acquire a lock and set newPostShuffleRDDs and estimated.
    +      this.synchronized {
    --- End diff --
    
    After second thought, I think it makes sense to just use the synchronized to guard the entire method. The code will be easier to understand.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42962038
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala ---
    @@ -465,6 +465,7 @@ private[hive] object TestHiveContext {
       val overrideConfs: Map[String, String] =
         Map(
           // Fewer shuffle partitions to speed up testing.
    -      SQLConf.SHUFFLE_PARTITIONS.key -> "5"
    +      SQLConf.SHUFFLE_PARTITIONS.key -> "10",
    +      SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "65536"
    --- End diff --
    
    These changes is for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152886455
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152352253
  
    **[Test build #44638 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44638/consoleFull)** for PR 9276 at commit [`197b63a`](https://github.com/apache/spark/commit/197b63ae5277991a004fa5f19943ec1605bb4141).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `class HasSolver(Params):`\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152089755
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44561/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152016613
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152073172
  
    **[Test build #44561 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44561/consoleFull)** for PR 9276 at commit [`2130068`](https://github.com/apache/spark/commit/2130068157c25b54b502d4dcf7a789c6f9e688d6).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152886456
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44781/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152915853
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719923
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    --- End diff --
    
    Should this be a set instead of an array buffer in order to guard against double-registration?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719970
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    --- End diff --
    
    `<` instead of `<=`?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577951
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    --- End diff --
    
    Done


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43473922
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    --- End diff --
    
    Can you comment on thread safety? Multiple threads read from the postShuffleRDDs at the same time right? But this happens after one batch insert?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153147316
  
    **[Test build #44818 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44818/consoleFull)** for PR 9276 at commit [`51ca17e`](https://github.com/apache/spark/commit/51ca17e8524457857fe09d9174fdc60af51f8e35).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153282386
  
    Let's merge this now and post-hoc review in more detail 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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151041904
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44342/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43722916
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala ---
    @@ -0,0 +1,479 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.test.TestSQLContext
    +import org.apache.spark.sql._
    +import org.apache.spark.{SparkFunSuite, SparkContext, SparkConf, MapOutputStatistics}
    +
    +class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var originalActiveSQLContext: Option[SQLContext] = _
    +  private var originalInstantiatedSQLContext: Option[SQLContext] = _
    +
    +  override protected def beforeAll(): Unit = {
    +    originalActiveSQLContext = SQLContext.getActiveContextOption()
    +    originalInstantiatedSQLContext = SQLContext.getInstantiatedContextOption()
    +
    +    SQLContext.clearActive()
    +    originalInstantiatedSQLContext.foreach(ctx => SQLContext.clearInstantiatedContext(ctx))
    +  }
    +
    +  override protected def afterAll(): Unit = {
    +    // Set these states back.
    --- End diff --
    
    `super.afterAll()`?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152324680
  
    **[Test build #44632 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44632/consoleFull)** for PR 9276 at commit [`f97ea39`](https://github.com/apache/spark/commit/f97ea3900ad3308f42a698a67a55d72fbbdaaa5c).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153106533
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151694675
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720616
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    Ah, so by "pre-shuffle" you're referring to the number of partitions output from `ShuffledRowRDD` and not the number of partitions present on the map-side? Just want to clarify terminology.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153106415
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43722910
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala ---
    @@ -0,0 +1,479 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.test.TestSQLContext
    +import org.apache.spark.sql._
    +import org.apache.spark.{SparkFunSuite, SparkContext, SparkConf, MapOutputStatistics}
    +
    +class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var originalActiveSQLContext: Option[SQLContext] = _
    +  private var originalInstantiatedSQLContext: Option[SQLContext] = _
    +
    +  override protected def beforeAll(): Unit = {
    --- End diff --
    
    Should this call `super.beforeAll()`?


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42962041
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -63,5 +63,6 @@ private[sql] object TestSQLContext {
       val overrideConfs: Map[String, String] =
         Map(
           // Fewer shuffle partitions to speed up testing.
    -      SQLConf.SHUFFLE_PARTITIONS.key -> "5")
    +      SQLConf.SHUFFLE_PARTITIONS.key -> "10",
    +      SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "65536")
    --- End diff --
    
    These changes is for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153237175
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-191440330
  
    No it is not documented. Right now, it is mainly for people who are interested in experimenting it. There are still work that needs to be done to make it support more cases.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152089722
  
    **[Test build #44561 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44561/consoleFull)** for PR 9276 at commit [`2130068`](https://github.com/apache/spark/commit/2130068157c25b54b502d4dcf7a789c6f9e688d6).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152940852
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44792/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152352371
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44638/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152789062
  
    **[Test build #44748 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44748/consoleFull)** for PR 9276 at commit [`2d1f262`](https://github.com/apache/spark/commit/2d1f262efb5b433eab430c20027838fddc427ac7).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153114509
  
    **[Test build #44818 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44818/consoleFull)** for PR 9276 at commit [`51ca17e`](https://github.com/apache/spark/commit/51ca17e8524457857fe09d9174fdc60af51f8e35).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152303684
  
    The last three commits are just for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152793965
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152007276
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577777
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -129,8 +143,29 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
         }
       }
     
    -  protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") {
    +  override protected def doPrepare(): Unit = {
    +    // If an ExchangeCoordinator is needed, we register this Exchange operator
    +    // to the coordinator when we do prepare. It is important to make sure
    +    // we register this operator right before the execution instead of register it
    +    // in the constructor.
    --- End diff --
    
    Done


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151037501
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43347860
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -202,12 +286,103 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
       private def canonicalPartitioning(requiredDistribution: Distribution): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    -      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions)
    +      case ClusteredDistribution(clustering) =>
    +        HashPartitioning(clustering, numPreShufflePartitions)
    +      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPreShufflePartitions)
           case dist => sys.error(s"Do not know how to satisfy distribution $dist")
         }
       }
     
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val needsCoordinator =
    +      if (children.exists(_.isInstanceOf[Exchange])) {
    +        // Right now, ExchangeCoordinator only support HashPartitionings.
    +        children.forall {
    +          case e @ Exchange(hash: HashPartitioning, _, _) => true
    +          case child =>
    +            child.outputPartitioning match {
    +              case hash: HashPartitioning => true
    +              case collection: PartitioningCollection =>
    +                collection.partitionings.exists(_.isInstanceOf[HashPartitioning])
    +              case _ => false
    +            }
    +        }
    +      } else {
    +        // In this case, although we do not have Exchange operators, we may still need to
    +        // shuffle data when we have more than one children because data generated by
    +        // these children may not be partitioned in the same way.
    +        // Please see the comment in withCoordinator for more details.
    +        val supportsDistribution =
    +          requiredChildDistributions.forall(_.isInstanceOf[ClusteredDistribution])
    +        children.length > 1 && supportsDistribution
    +      }
    +
    +    val withCoordinator =
    +      if (adaptiveExecutionEnabled && needsCoordinator) {
    +        val coordinator =
    +          new ExchangeCoordinator(
    +            children.length,
    +            targetPostShuffleInputSize,
    +            minNumPostShufflePartitions)
    +        children.zip(requiredChildDistributions).map {
    +          case (e: Exchange, _) =>
    +            // This child is an Exchange, we need to add the coordinator.
    +            e.copy(coordinator = Some(coordinator))
    +          case (child, distribution) =>
    +            // If this child is not an Exchange, we need to add an Exchange for now.
    +            // Ideally, we can try to avoid this Exchange. However, when we reach here,
    +            // there are at least two children operators (because if there is a single child
    +            // and we can avoid Exchange, this method will not be called). Although we can
    +            // make two children have the same number of post-shuffle partitions. Their
    +            // numbers of pre-shuffle partitions may be different. For example, let's say
    +            // we have the following plan
    +            //         Join
    +            //         /  \
    +            //       Agg  Exchange
    +            //       /      \
    +            //    Exchange  t2
    +            //      /
    +            //     t1
    +            // In this case, because a post-shuffle partition can include multiple pre-shuffle
    +            // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes
    +            // after shuffle. So, even we can use the child Exchange operator of the Join to
    +            // have a number of post-shuffle partitions that matches the number of partitions of
    +            // Agg, we cannot say these two children are partitioned in the same way.
    +            // Here is another case
    +            //         Join
    +            //         /  \
    +            //       Agg1  Agg2
    +            //       /      \
    +            //   Exchange1  Exchange2
    +            //       /       \
    +            //      t1       t2
    +            // In this case, two Aggs shuffle data with the same column of the join condition.
    +            // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same
    +            // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2
    +            // post-shuffle partitions. However, Agg1 fetches those pre-shuffle partitions by
    +            // using a partitionStartIndices [0, 3]. But, Agg1 fetches those pre-shuffle
    +            // partitions by using another partitionStartIndices [0, 4]. So, Agg1 and Agg2
    +            // are actually not partitioned in the same way. So, we need to add Exchanges at here.
    +            //
    +            // It will be great to introduce a new Partitioning to represent the post-shuffle
    +            // partitions when one post-shuffle partition includes multiple pre-shuffle partitions.
    +
    +            // Because originally we do not have an Exchange operator, we can just use this child
    +            // operator's outputPartitioning to shuffle data.
    +            val targetPartitioning = canonicalPartitioning(distribution)
    +            assert(targetPartitioning.isInstanceOf[HashPartitioning])
    +            Exchange(targetPartitioning, child, Some(coordinator))
    --- End diff --
    
    It is not a good idea. It will shuffle data more times and make certain queries with multiple joins inefficient. Right now, it is mainly to make sure we can generate correct results. Once we can generate a DAG of query fragments (based on https://github.com/apache/spark/pull/9039). I think this problem can be addressed.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151717781
  
    **[Test build #44487 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44487/consoleFull)** for PR 9276 at commit [`27af247`](https://github.com/apache/spark/commit/27af24735121ec97f73626dc9e9c08b2cdcd78c6).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152309854
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152307569
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44630/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577781
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -202,12 +286,103 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
       private def canonicalPartitioning(requiredDistribution: Distribution): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    -      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions)
    +      case ClusteredDistribution(clustering) =>
    +        HashPartitioning(clustering, numPreShufflePartitions)
    +      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPreShufflePartitions)
           case dist => sys.error(s"Do not know how to satisfy distribution $dist")
         }
       }
     
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val needsCoordinator =
    --- End diff --
    
    yes.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152329755
  
    **[Test build #44638 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44638/consoleFull)** for PR 9276 at commit [`197b63a`](https://github.com/apache/spark/commit/197b63ae5277991a004fa5f19943ec1605bb4141).


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151671089
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#discussion_r42961906
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/test/TestSQLContext.scala ---
    @@ -63,5 +63,5 @@ private[sql] object TestSQLContext {
       val overrideConfs: Map[String, String] =
         Map(
           // Fewer shuffle partitions to speed up testing.
    -      SQLConf.SHUFFLE_PARTITIONS.key -> "5")
    +      SQLConf.SHUFFLE_PARTITIONS.key -> "10")
    --- End diff --
    
    This change is for testing purpose.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152940621
  
    **[Test build #44792 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44792/consoleFull)** for PR 9276 at commit [`36b6dbc`](https://github.com/apache/spark/commit/36b6dbc042b5a127ba230b6bf425259ae7e5a7ac).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Corr(`\n  * `case class Corr(left: Expression, right: Expression)`\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151037339
  
    **[Test build #44342 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44342/consoleFull)** for PR 9276 at commit [`3abf78f`](https://github.com/apache/spark/commit/3abf78fdd66f11f2bc6924714eb721a8c32b3bb2).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152089754
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153284506
  
    ok. I am merging it now. Will have a follow-up pr to address @JoshRosen's comments.


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

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


[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153282979
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43334997
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -202,12 +286,103 @@ private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[
       private def canonicalPartitioning(requiredDistribution: Distribution): Partitioning = {
         requiredDistribution match {
           case AllTuples => SinglePartition
    -      case ClusteredDistribution(clustering) => HashPartitioning(clustering, numPartitions)
    -      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPartitions)
    +      case ClusteredDistribution(clustering) =>
    +        HashPartitioning(clustering, numPreShufflePartitions)
    +      case OrderedDistribution(ordering) => RangePartitioning(ordering, numPreShufflePartitions)
           case dist => sys.error(s"Do not know how to satisfy distribution $dist")
         }
       }
     
    +  private def withExchangeCoordinator(
    +      children: Seq[SparkPlan],
    +      requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = {
    +    val needsCoordinator =
    +      if (children.exists(_.isInstanceOf[Exchange])) {
    +        // Right now, ExchangeCoordinator only support HashPartitionings.
    +        children.forall {
    +          case e @ Exchange(hash: HashPartitioning, _, _) => true
    +          case child =>
    +            child.outputPartitioning match {
    +              case hash: HashPartitioning => true
    +              case collection: PartitioningCollection =>
    +                collection.partitionings.exists(_.isInstanceOf[HashPartitioning])
    +              case _ => false
    +            }
    +        }
    +      } else {
    +        // In this case, although we do not have Exchange operators, we may still need to
    +        // shuffle data when we have more than one children because data generated by
    +        // these children may not be partitioned in the same way.
    +        // Please see the comment in withCoordinator for more details.
    +        val supportsDistribution =
    +          requiredChildDistributions.forall(_.isInstanceOf[ClusteredDistribution])
    +        children.length > 1 && supportsDistribution
    +      }
    +
    +    val withCoordinator =
    +      if (adaptiveExecutionEnabled && needsCoordinator) {
    +        val coordinator =
    +          new ExchangeCoordinator(
    +            children.length,
    +            targetPostShuffleInputSize,
    +            minNumPostShufflePartitions)
    +        children.zip(requiredChildDistributions).map {
    +          case (e: Exchange, _) =>
    +            // This child is an Exchange, we need to add the coordinator.
    +            e.copy(coordinator = Some(coordinator))
    +          case (child, distribution) =>
    +            // If this child is not an Exchange, we need to add an Exchange for now.
    +            // Ideally, we can try to avoid this Exchange. However, when we reach here,
    +            // there are at least two children operators (because if there is a single child
    +            // and we can avoid Exchange, this method will not be called). Although we can
    +            // make two children have the same number of post-shuffle partitions. Their
    +            // numbers of pre-shuffle partitions may be different. For example, let's say
    +            // we have the following plan
    +            //         Join
    +            //         /  \
    +            //       Agg  Exchange
    +            //       /      \
    +            //    Exchange  t2
    +            //      /
    +            //     t1
    +            // In this case, because a post-shuffle partition can include multiple pre-shuffle
    +            // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes
    +            // after shuffle. So, even we can use the child Exchange operator of the Join to
    +            // have a number of post-shuffle partitions that matches the number of partitions of
    +            // Agg, we cannot say these two children are partitioned in the same way.
    +            // Here is another case
    +            //         Join
    +            //         /  \
    +            //       Agg1  Agg2
    +            //       /      \
    +            //   Exchange1  Exchange2
    +            //       /       \
    +            //      t1       t2
    +            // In this case, two Aggs shuffle data with the same column of the join condition.
    +            // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same
    +            // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2
    +            // post-shuffle partitions. However, Agg1 fetches those pre-shuffle partitions by
    +            // using a partitionStartIndices [0, 3]. But, Agg1 fetches those pre-shuffle
    +            // partitions by using another partitionStartIndices [0, 4]. So, Agg1 and Agg2
    +            // are actually not partitioned in the same way. So, we need to add Exchanges at here.
    +            //
    +            // It will be great to introduce a new Partitioning to represent the post-shuffle
    +            // partitions when one post-shuffle partition includes multiple pre-shuffle partitions.
    +
    +            // Because originally we do not have an Exchange operator, we can just use this child
    +            // operator's outputPartitioning to shuffle data.
    +            val targetPartitioning = canonicalPartitioning(distribution)
    +            assert(targetPartitioning.isInstanceOf[HashPartitioning])
    +            Exchange(targetPartitioning, child, Some(coordinator))
    --- End diff --
    
    Are you sure this is a good idea? Adding exchanges seems very expensive.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151674264
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152793863
  
    **[Test build #44748 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44748/consoleFull)** for PR 9276 at commit [`2d1f262`](https://github.com/apache/spark/commit/2d1f262efb5b433eab430c20027838fddc427ac7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153237721
  
    **[Test build #44879 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44879/consoleFull)** for PR 9276 at commit [`4a99729`](https://github.com/apache/spark/commit/4a997295a3e68ba9b137e9f65fac89c2cf4a0cfc).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720726
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    +        math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize)
    +
    +      case None => advisoryTargetPostShuffleInputSize
    +    }
    +
    +    logInfo(
    +      s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " +
    +      s"targetPostShuffleInputSize $targetPostShuffleInputSize.")
    +
    +    // Make sure we do get the same number of pre-shuffle partitions for those stages.
    +    val distinctNumPreShufflePartitions =
    +      mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct
    +    assert(
    +      distinctNumPreShufflePartitions.length == 1,
    +      "There should be only one distinct value of the number pre-shuffle partitions " +
    --- End diff --
    
    oh, pre-shuffle partitions mean those map output partitions (these partitions will be fetched by the ShuffleReader). Partitions output from `ShuffledRowRDD` are actually post-shuffle partitions.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720557
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    --- End diff --
    
    yeah, you are right.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152008510
  
    @JoshRosen  I changed `InnerJoinSuite` to make our physical join operator do not use the same SparkPlan as its both left and right children (https://github.com/yhuai/spark/commit/c2463da9499b1b12c1f7f59fb9ba15d615e2123e).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152793967
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44748/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152886380
  
    **[Test build #44781 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44781/consoleFull)** for PR 9276 at commit [`890521e`](https://github.com/apache/spark/commit/890521e54d81f8ef8172a43fff50f824a7fec1be).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151717426
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151034664
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152785671
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43577949
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    --- End diff --
    
    Done


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43722957
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala ---
    @@ -0,0 +1,479 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import org.scalatest.BeforeAndAfterAll
    +
    +import org.apache.spark.sql.functions._
    +import org.apache.spark.sql.test.TestSQLContext
    +import org.apache.spark.sql._
    +import org.apache.spark.{SparkFunSuite, SparkContext, SparkConf, MapOutputStatistics}
    +
    +class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll {
    +
    +  private var originalActiveSQLContext: Option[SQLContext] = _
    +  private var originalInstantiatedSQLContext: Option[SQLContext] = _
    +
    +  override protected def beforeAll(): Unit = {
    --- End diff --
    
    Seems we are fine since `SparkFunSuite` does not extend `BeforeAndAfterAll`.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151671117
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43473235
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    --- End diff --
    
    Will add a `toString` method to this class.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152072530
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152307566
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151035067
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152016614
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44544/
    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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152312694
  
    **[Test build #44632 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44632/consoleFull)** for PR 9276 at commit [`f97ea39`](https://github.com/apache/spark/commit/f97ea3900ad3308f42a698a67a55d72fbbdaaa5c).


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43473446
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,251 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with two parameters, `numExchanges` and
    + * `targetPostShuffleInputSize`. `numExchanges` is used to indicated that how many [[Exchange]]s
    + * that will be registered to this coordinator. So, when we start to do any actual work, we have
    + * a way to make sure that we have got expected number of [[Exchange]]s.
    + * `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's input data size.
    + * With this parameter, we can estimate the number of post-shuffle partitions. This parameter
    + * is configured through `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[RDD]]. If this coordinator has
    + *    made the decision on how to shuffle data, this [[Exchange]] will immediately get its
    + *    corresponding post-shuffle [[RDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[RDD]]s for all registered [[Exchange]]s.
    + *    So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can lookup the
    + *    corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + *
    + * If `minNumPostShufflePartitions` is defined. This ExchangeCoordinator will try to enforce
    + * the minimal number of post-shuffle partitions to this number.
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map that is used to lookup the post-shuffle RDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean indicates if this coordinator has made decision on how to shuffle data.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // At here, we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    --- End diff --
    
    "At here" -> "If"?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-191230900
  
    Is `spark.sql.adaptive.enabled` documented somewhere? It's not in http://spark.apache.org/docs/1.6.0/configuration.html.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152324775
  
    Merged build finished. 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r49286654
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    --- End diff --
    
    According to the implementation, we can't guarantee to satisfy `minNumPostShufflePartitions` right?


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153248463
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720590
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala ---
    @@ -181,7 +215,54 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una
             }
           }
         }
    -    new ShuffledRowRDD(rddWithPartitionIds, serializer, part.numPartitions)
    +
    +    // Now, we manually create a ShuffleDependency. Because pairs in rddWithPartitionIds
    +    // are in the form of (partitionId, row) and every partitionId is in the expected range
    +    // [0, part.numPartitions - 1]. The partitioner of this is a PartitionIdPassthrough.
    +    val dependency =
    +      new ShuffleDependency[Int, InternalRow, InternalRow](
    +        rddWithPartitionIds,
    +        new PartitionIdPassthrough(part.numPartitions),
    +        Some(serializer))
    +
    +    dependency
    +  }
    +
    +  /**
    +   * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset.
    +   * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional
    +   * partition start indices array. If this optional array is defined, the returned
    +   * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array.
    +   */
    +  private[sql] def preparePostShuffleRDD(
    +      shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow],
    +      specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = {
    +    // If an array of partition start indices is provided, we need to use this array
    +    // to create the ShuffledRowRDD. Also, we need to update newPartitioning to
    +    // update the number of post-shuffle partitions.
    +    specifiedPartitionStartIndices.foreach { indices =>
    +      assert(newPartitioning.isInstanceOf[HashPartitioning])
    +      newPartitioning = newPartitioning.withNumPartitions(indices.length)
    +    }
    +    new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices)
    +  }
    +
    +  protected override def doExecute(): RDD[InternalRow] = attachTree(this , "execute") {
    +    coordinator match {
    +      case Some(exchangeCoordinator) =>
    +        val shuffleRDD = exchangeCoordinator.postShuffleRDD(this)
    +        assert(shuffleRDD.partitions.length == newPartitioning.numPartitions)
    +        shuffleRDD
    +      case None =>
    +        val shuffleDependency = prepareShuffleDependency()
    +        preparePostShuffleRDD(shuffleDependency)
    +    }
    +  }
    +}
    +
    +object Exchange {
    +  def apply(newPartitioning: Partitioning, child: SparkPlan): Exchange = {
    +    Exchange(newPartitioning, child, None: Option[ExchangeCoordinator])
    --- End diff --
    
    sure.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151719604
  
    **[Test build #44487 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44487/consoleFull)** for PR 9276 at commit [`27af247`](https://github.com/apache/spark/commit/27af24735121ec97f73626dc9e9c08b2cdcd78c6).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:\n  * `case class Exchange(`\n  * `class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: Array[Int])`\n


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152007217
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-153248403
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720001
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    --- End diff --
    
    Why 16?


---
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 issue #9276: [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add an Exchange...

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

    https://github.com/apache/spark/pull/9276
  
    @yhuai , could you please let us know is there any known issues / limitation with this feature ? Has this feature been tested under some large jobs ? 
    
    We are also considering automatical determining shuffle partitions, and happened to see this PR, and therefore interested in exploring this feature a little bit to see if we could productionize it for all jobs (by default).


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151041862
  
    **[Test build #44342 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/44342/consoleFull)** for PR 9276 at commit [`3abf78f`](https://github.com/apache/spark/commit/3abf78fdd66f11f2bc6924714eb721a8c32b3bb2).
     * 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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152787723
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720666
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    --- End diff --
    
    If that is the case, I guess we should expose that and fix the double registration problem, right? In `doEstimationIfNecessary`, we have an assert `assert(exchanges.length == numExchanges)`. 


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151035063
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152915827
  
     Merged build triggered.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

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


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r49287822
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    --- End diff --
    
    Yes, you are right. It is advisory.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152328909
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151034685
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152787654
  
    test this please


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

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


[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#issuecomment-152787732
  
    Merged build started.


---
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-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43719859
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    --- End diff --
    
    Add an `@GuardedBy("this")`?


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

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


[GitHub] spark pull request: [SPARK-9858][SPARK-9859][SPARK-9861][SQL] Add ...

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

    https://github.com/apache/spark/pull/9276#discussion_r43720550
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/ExchangeCoordinator.scala ---
    @@ -0,0 +1,260 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *    http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.spark.sql.execution
    +
    +import java.util.{Map => JMap, HashMap => JHashMap}
    +
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, SimpleFutureAction, ShuffleDependency, MapOutputStatistics}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.sql.catalyst.InternalRow
    +
    +/**
    + * A coordinator used to determines how we shuffle data between stages generated by Spark SQL.
    + * Right now, the work of this coordinator is to determine the number of post-shuffle partitions
    + * for a stage that needs to fetch shuffle data from one or multiple stages.
    + *
    + * A coordinator is constructed with three parameters, `numExchanges`,
    + * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`.
    + *  - `numExchanges` is used to indicated that how many [[Exchange]]s that will be registered to
    + *    this coordinator. So, when we start to do any actual work, we have a way to make sure that
    + *    we have got expected number of [[Exchange]]s.
    + *  - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's
    + *    input data size. With this parameter, we can estimate the number of post-shuffle partitions.
    + *    This parameter is configured through
    + *    `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`.
    + *  - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator
    + *    will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle
    + *    partitions.
    + *
    + * The workflow of this coordinator is described as follows:
    + *  - Before the execution of a [[SparkPlan]], for an [[Exchange]] operator,
    + *    if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator.
    + *    This happens in the `doPrepare` method.
    + *  - Once we start to execute a physical plan, an [[Exchange]] registered to this coordinator will
    + *    call `postShuffleRDD` to get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *    If this coordinator has made the decision on how to shuffle data, this [[Exchange]] will
    + *    immediately get its corresponding post-shuffle [[ShuffledRowRDD]].
    + *  - If this coordinator has not made the decision on how to shuffle data, it will ask those
    + *    registered [[Exchange]]s to submit their pre-shuffle stages. Then, based on the the size
    + *    statistics of pre-shuffle partitions, this coordinator will determine the number of
    + *    post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices
    + *    to a single post-shuffle partition whenever necessary.
    + *  - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered
    + *    [[Exchange]]s. So, when an [[Exchange]] calls `postShuffleRDD`, this coordinator can
    + *    lookup the corresponding [[RDD]].
    + *
    + * The strategy used to determine the number of post-shuffle partitions is described as follows.
    + * To determine the number of post-shuffle partitions, we have a target input size for a
    + * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages
    + * corresponding to the registered [[Exchange]]s, we will do a pass of those statistics and
    + * pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until
    + * the size of a post-shuffle partition is equal or greater than the target size.
    + * For example, we have two stages with the following pre-shuffle partition size statistics:
    + * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB]
    + * stage 2: [10 MB,  10 MB, 70 MB,  5 MB, 5 MB]
    + * assuming the target input size is 128 MB, we will have three post-shuffle partitions,
    + * which are:
    + *  - post-shuffle partition 0: pre-shuffle partition 0 and 1
    + *  - post-shuffle partition 1: pre-shuffle partition 2
    + *  - post-shuffle partition 2: pre-shuffle partition 3 and 4
    + */
    +private[sql] class ExchangeCoordinator(
    +    numExchanges: Int,
    +    advisoryTargetPostShuffleInputSize: Long,
    +    minNumPostShufflePartitions: Option[Int] = None)
    +  extends Logging {
    +
    +  // The registered Exchange operators.
    +  private[this] val exchanges = ArrayBuffer[Exchange]()
    +
    +  // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator.
    +  private[this] val postShuffleRDDs: JMap[Exchange, ShuffledRowRDD] =
    +    new JHashMap[Exchange, ShuffledRowRDD](numExchanges)
    +
    +  // A boolean that indicates if this coordinator has made decision on how to shuffle data.
    +  // This variable will only be updated by doEstimationIfNecessary, which is protected by
    +  // synchronized.
    +  @volatile private[this] var estimated: Boolean = false
    +
    +  /**
    +   * Registers an [[Exchange]] operator to this coordinator. This method is only allowed to be
    +   * called in the `doPrepare` method of an [[Exchange]] operator.
    +   */
    +  def registerExchange(exchange: Exchange): Unit = synchronized {
    +    exchanges += exchange
    +  }
    +
    +  def isEstimated: Boolean = estimated
    +
    +  /**
    +   * Estimates partition start indices for post-shuffle partitions based on
    +   * mapOutputStatistics provided by all pre-shuffle stages.
    +   */
    +  private[sql] def estimatePartitionStartIndices(
    +      mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = {
    +    // If we have mapOutputStatistics.length <= numExchange, it is because we do not submit
    +    // a stage when the number of partitions of this dependency is 0.
    +    assert(mapOutputStatistics.length <= numExchanges)
    +
    +    // If minNumPostShufflePartitions is defined, it is possible that we need to use a
    +    // value less than advisoryTargetPostShuffleInputSize as the target input size of
    +    // a post shuffle task.
    +    val targetPostShuffleInputSize = minNumPostShufflePartitions match {
    +      case Some(numPartitions) =>
    +        val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum
    +        // The max at here is to make sure that when we have an empty table, we
    +        // only have a single post-shuffle partition.
    +        val maxPostShuffleInputSize =
    +          math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16)
    --- End diff --
    
    There is no particular reason. I just need a small number to prevent it from being 0.


---
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-9858][SPARK-9859][SPARK-9861][SQL][WIP]...

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

    https://github.com/apache/spark/pull/9276#issuecomment-151717410
  
     Merged build triggered.


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