New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-19426][SQL] Custom coalescer for Dataset #18861
Changes from 2 commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -17,12 +17,12 @@ | |
|
||
package org.apache.spark.sql.catalyst.plans.logical | ||
|
||
import org.apache.spark.rdd.PartitionCoalescer | ||
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation | ||
import org.apache.spark.sql.catalyst.catalog.CatalogTable | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression | ||
import org.apache.spark.sql.catalyst.plans._ | ||
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation._ | ||
import org.apache.spark.sql.types._ | ||
import org.apache.spark.util.Utils | ||
import org.apache.spark.util.random.RandomSampler | ||
|
@@ -752,6 +752,16 @@ case class Repartition(numPartitions: Int, shuffle: Boolean, child: LogicalPlan) | |
require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Add a new require here?
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok |
||
} | ||
|
||
/** | ||
* Returns a new RDD that has at most `numPartitions` partitions. This behavior can be modified by | ||
* supplying a `PartitionCoalescer` to control the behavior of the partitioning. | ||
*/ | ||
case class PartitionCoalesce(numPartitions: Int, coalescer: PartitionCoalescer, child: LogicalPlan) | ||
extends UnaryNode { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Adding new logical nodes also needs the updates in multiple different components. (e.g., Optimizer). Is that possible to reuse the existing node There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yea, I think so. I'll try and plz give me days to do so. |
||
require(numPartitions > 0, s"Number of partitions ($numPartitions) must be positive.") | ||
override def output: Seq[Attribute] = child.output | ||
} | ||
|
||
/** | ||
* This method repartitions data using [[Expression]]s into `numPartitions`, and receives | ||
* information about the number of partitions during execution. Used when a specific ordering or | ||
|
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -20,14 +20,13 @@ package org.apache.spark.sql.execution | |
import scala.concurrent.{ExecutionContext, Future} | ||
import scala.concurrent.duration.Duration | ||
|
||
import org.apache.spark.{InterruptibleIterator, SparkException, TaskContext} | ||
import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} | ||
import org.apache.spark.{InterruptibleIterator, TaskContext} | ||
import org.apache.spark.rdd.{EmptyRDD, PartitionCoalescer, PartitionwiseSampledRDD, RDD} | ||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions._ | ||
import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, ExpressionCanonicalizer} | ||
import org.apache.spark.sql.catalyst.plans.physical._ | ||
import org.apache.spark.sql.execution.metric.SQLMetrics | ||
import org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates | ||
import org.apache.spark.sql.types.LongType | ||
import org.apache.spark.util.ThreadUtils | ||
import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} | ||
|
@@ -571,7 +570,8 @@ case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan { | |
* current upstream partitions will be executed in parallel (per whatever | ||
* the current partitioning is). | ||
*/ | ||
case class CoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode { | ||
case class CoalesceExec(numPartitions: Int, child: SparkPlan, coalescer: Option[PartitionCoalescer]) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you add the parm description of There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. ok! |
||
extends UnaryExecNode { | ||
override def output: Seq[Attribute] = child.output | ||
|
||
override def outputPartitioning: Partitioning = { | ||
|
@@ -580,7 +580,7 @@ case class CoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecN | |
} | ||
|
||
protected override def doExecute(): RDD[InternalRow] = { | ||
child.execute().coalesce(numPartitions, shuffle = false) | ||
child.execute().coalesce(numPartitions, shuffle = false, coalescer) | ||
} | ||
} | ||
|
||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
All the above changes are not related to this PR, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yea, I just left the changes of the original author (probably refactoring stuffs?) ..., better remove this?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I am fine about this, but it might confuse the others. Maybe just remove them in this PR? You can submit a separate PR later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
ok, I'll drop these from this pr.