szehon-ho commented on code in PR #54330:
URL: https://github.com/apache/spark/pull/54330#discussion_r2825236785
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala:
##########
@@ -346,43 +348,85 @@ case class CoalescedHashPartitioning(from:
HashPartitioning, partitions: Seq[Coa
}
/**
- * Represents a partitioning where rows are split across partitions based on
transforms defined
- * by `expressions`. `partitionValues`, if defined, should contain value of
partition key(s) in
- * ascending order, after evaluated by the transforms in `expressions`, for
each input partition.
- * In addition, its length must be the same as the number of Spark partitions
(and thus is a 1-1
- * mapping), and each row in `partitionValues` must be unique.
+ * Represents a partitioning where rows are split across partitions based on
transforms defined by
+ * `expressions`. `partitionKeys`, should contain value of partition key(s) in
ascending order,
+ * after evaluated by the transforms in `expressions`, for each input
partition.
+ * `partitionKeys` might not be unique when this partitioning is returned from
a data source, but
+ * the `GroupPartitionsExec` operator can group partitions with the same key
and so make
+ * `partitionKeys` unique.
*
- * The `originalPartitionValues`, on the other hand, are partition values from
the original input
+ * The `originalPartitionKeys`, on the other hand, are partition values from
the original input
Review Comment:
nit: looks like 'originalPartitionKeys' are a copy of 'partitionKeys' before
any grouping is applied, . Can we clarify the comment as its not clear from it
currently
##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala:
##########
@@ -346,43 +348,85 @@ case class CoalescedHashPartitioning(from:
HashPartitioning, partitions: Seq[Coa
}
/**
- * Represents a partitioning where rows are split across partitions based on
transforms defined
- * by `expressions`. `partitionValues`, if defined, should contain value of
partition key(s) in
- * ascending order, after evaluated by the transforms in `expressions`, for
each input partition.
- * In addition, its length must be the same as the number of Spark partitions
(and thus is a 1-1
- * mapping), and each row in `partitionValues` must be unique.
+ * Represents a partitioning where rows are split across partitions based on
transforms defined by
+ * `expressions`. `partitionKeys`, should contain value of partition key(s) in
ascending order,
Review Comment:
nit: seems we lost 'if defined', so the comma doesn't make sense anymore
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala:
##########
@@ -56,94 +91,60 @@ class DataSourceRDD(
}
override def compute(split: Partition, context: TaskContext):
Iterator[InternalRow] = {
-
- val iterator = new Iterator[Object] {
- private val inputPartitions = castPartition(split).inputPartitions
- private var currentIter: Option[Iterator[Object]] = None
- private var currentIndex: Int = 0
-
- private val partitionMetricCallback = new
PartitionMetricCallback(customMetrics)
-
- // In case of early stopping before consuming the entire iterator,
- // we need to do one more metric update at the end of the task.
- context.addTaskCompletionListener[Unit] { _ =>
- partitionMetricCallback.execute()
- }
-
- override def hasNext: Boolean = currentIter.exists(_.hasNext) ||
advanceToNextIter()
-
- override def next(): Object = {
- if (!hasNext) throw new NoSuchElementException("No more elements")
- currentIter.get.next()
+ castPartition(split).inputPartition.iterator.flatMap { inputPartition =>
+ val (iter, reader) = if (columnarReads) {
+ val batchReader =
partitionReaderFactory.createColumnarReader(inputPartition)
+ val iter = new MetricsBatchIterator(
+ new PartitionIterator[ColumnarBatch](batchReader, customMetrics),
readerStateThreadLocal)
+ (iter, batchReader)
+ } else {
+ val rowReader = partitionReaderFactory.createReader(inputPartition)
+ val iter = new MetricsRowIterator(
+ new PartitionIterator[InternalRow](rowReader, customMetrics),
readerStateThreadLocal)
+ (iter, rowReader)
}
- private def advanceToNextIter(): Boolean = {
- if (currentIndex >= inputPartitions.length) {
- false
- } else {
- val inputPartition = inputPartitions(currentIndex)
- currentIndex += 1
-
- // TODO: SPARK-25083 remove the type erasure hack in data source scan
- val (iter, reader) = if (columnarReads) {
- val batchReader =
partitionReaderFactory.createColumnarReader(inputPartition)
- val iter = new MetricsBatchIterator(
- new PartitionIterator[ColumnarBatch](batchReader, customMetrics))
- (iter, batchReader)
- } else {
- val rowReader = partitionReaderFactory.createReader(inputPartition)
- val iter = new MetricsRowIterator(
- new PartitionIterator[InternalRow](rowReader, customMetrics))
- (iter, rowReader)
+ // Add completion listener only once per thread (null means no listener
added yet)
+ val readerState = readerStateThreadLocal.get()
+ if (readerState == null) {
+ context.addTaskCompletionListener[Unit] { _ =>
+ // Use the reader and iterator from ThreadLocal (the last ones
created in this thread)
+ val readerState = readerStateThreadLocal.get()
+ if (readerState != null) {
+ // In case of early stopping before consuming the entire iterator,
+ // we need to do one more metric update at the end of the task.
+ CustomMetrics.updateMetrics(
+ readerState.reader.currentMetricsValues.toImmutableArraySeq,
customMetrics)
+ readerState.iterator.forceUpdateMetrics()
+ readerState.reader.close()
}
-
- // Once we advance to the next partition, update the metric callback
for early finish
- val previousMetrics = partitionMetricCallback.advancePartition(iter,
reader)
- previousMetrics.foreach(reader.initMetricsValues)
-
- currentIter = Some(iter)
- hasNext
+ readerStateThreadLocal.remove()
}
+ } else {
+ readerState.metrics.foreach(reader.initMetricsValues)
Review Comment:
are we missing a close, maybe here to old readerState.reader?
cc @viirya i believe fixed a memory leak here to also take a look at the new
approach
##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/GroupPartitionsExec.scala:
##########
@@ -0,0 +1,220 @@
+/*
+ * 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.datasources.v2
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.Partition
+import org.apache.spark.rdd.{CoalescedRDD, PartitionCoalescer, PartitionGroup,
RDD}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.physical.{KeyedPartitioning,
Partitioning}
+import org.apache.spark.sql.catalyst.util.InternalRowComparableWrapper
+import org.apache.spark.sql.connector.catalog.functions.Reducer
+import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.types.DataType
+
+/**
+ * Physical operator that groups input partitions by their partition keys.
+ *
+ * This operator is used to coalesce partitions from bucketed/partitioned data
sources
+ * where multiple input partitions share the same partition key. It's commonly
used in
+ * storage-partitioned joins to align partitions from different sides of the
join.
+ *
+ * @param child The child plan providing bucketed/partitioned input
+ * @param joinKeyPositions Optional projection to select a subset of the
partitioning key
+ * for join compatibility (e.g., when join keys are a
subset of
+ * partition keys)
+ * @param commonPartitionKeys Optional sequence of expected partition key
values and their
+ * split counts, used for partially clustered data
+ * @param reducers Optional reducers to apply to partition keys for grouping
compatibility
+ * @param applyPartialClustering Whether to apply partial clustering for
skewed data
+ * @param replicatePartitions Whether to replicate partitions across multiple
keys
+ */
+case class GroupPartitionsExec(
+ child: SparkPlan,
+ joinKeyPositions: Option[Seq[Int]] = None,
+ commonPartitionKeys: Option[Seq[(InternalRow, Int)]] = None,
+ reducers: Option[Seq[Option[Reducer[_, _]]]] = None,
+ applyPartialClustering: Boolean = false,
+ replicatePartitions: Boolean = false
+ ) extends UnaryExecNode {
+
+ override def outputPartitioning: Partitioning = {
+ child.outputPartitioning match {
+ case p: Partitioning with Expression =>
+ p.transform {
+ case k: KeyedPartitioning =>
+ val projectedExpressions = projectExpressions(k.expressions)
+ val projectedDataTypes = projectedExpressions.map(_.dataType)
+ k.copy(expressions = projectedExpressions,
+ partitionKeys = groupedPartitions.map(_._1),
+ originalPartitionKeys = projectKeys(k.originalPartitionKeys,
projectedDataTypes))
+ }.asInstanceOf[Partitioning]
+ case o => o
+ }
+ }
+
+ private def projectExpressions(expressions: Seq[Expression]) = {
+ joinKeyPositions match {
+ case Some(projectionPositions) =>
+ projectionPositions.map(expressions)
+ case _ => expressions
+ }
+ }
+
+ private def projectKeys(keys: Seq[InternalRow], dataTypes: Seq[DataType]) = {
+ joinKeyPositions match {
+ case Some(projectionPositions) =>
+ keys.map(KeyedPartitioning.projectKey(_, projectionPositions,
dataTypes))
+ case _ => keys
+ }
+ }
+
+ /**
+ * Extracts the first KeyedPartitioning from the child's output partitioning.
+ * The child must have a KeyedPartitioning in its partitioning scheme.
+ */
+ lazy val firstKeyedPartitioning = {
+ child.outputPartitioning.asInstanceOf[Partitioning with
Expression].collectFirst {
+ case k: KeyedPartitioning => k
+ }.get
Review Comment:
nit: how about add .getOrElse(throw new SparkException("requires child with
KeyedPartitioning")) to be more clear when error happens
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]