-
Notifications
You must be signed in to change notification settings - Fork 28.4k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[SPARK-37377][SQL] Initial implementation of Storage-Partitioned Join
### What changes were proposed in this pull request? This PR introduces the initial implementation of Storage-Partitioned Join ([SPIP](https://docs.google.com/document/d/1foTkDSM91VxKgkEcBMsuAvEjNybjja-uHk-r3vtXWFE)). Changes: - `org.apache.spark.sql.connector.read.partitioning.Partitioning` currently is very limited (as mentioned in the SPIP), and cannot be extended to handle join cases. This PR completely replace it following the catalyst `Partitioning` interface, and added two concrete sub-classes: `KeyGroupedPartitioning` and `UnknownPartitioning`. This allows a V2 data source to report to Spark it's partition transform expressions, via `SupportsReportPartitioning` interface. - with the above change, `org.apache.spark.sql.connector.read.partitioning.Distribution` and `org.apache.spark.sql.connector.read.partitioning.ClusteredDistribution` now are replaced by classes with the same name in `org.apache.spark.sql.connector.distributions` package. Therefore, this PR marks the former two as deprecated. - `DataSourcePartitioning` used to be in `org.apache.spark.sql.execution.datasources.v2`. This moves it into package `org.apache.spark.sql.catalyst.plans.physical` and renames it to `KeyGroupedPartitioning`, so that it can be extended for more non-V2 use cases, such as Hive bucketing. In addition, it is also changed to accommodate the Storage-Partitioned Join feature. - a new expression type: `TransformExpression`, is introduced to bind syntactic partition transforms with their semantic meaning, represented by a V2 function. This expression is un-evaluable for now, and is used later in `EnsureRequirements` to check whether join children are compatible with each other. - a new optimizer rule: `V2ScanPartitioning`, is added to recognize `Scan`s implement `SupportsReportPartitioning`. If they do, this rule converts V2 partition transform expressions into their counterparts in catalyst, and annotate `DataSourceV2ScanRelation` with the result. These are later propagated into `DataSourceV2ScanExecBase`. - changes are made in `DataSourceV2ScanExecBase` to create `KeyGroupedPartitioning` for scan if 1) the scan is annotated with catalyst partition transform expressions, and 2) if all input splits implement `HasPartitionKey`. - A new config: `spark.sql.sources.v2.bucketing.enabled` is introduced to turn on or off the behavior. By default it is false. ### Why are the changes needed? Spark currently support bucketing in DataSource V1, but not in V2. This is the first step to support bucket join, and is general form, storage-partitioned join, for V2 data sources. In addition, the work here can potentially used to support Hive bucketing as well. Please check the SPIP for details. ### Does this PR introduce _any_ user-facing change? With the changes, a user can now: - have V2 data sources to report distribution and ordering to Spark on read path - Spark will recognize the distribution property and eliminate shuffle in join/aggregate/window, etc, when the source distribution matches the required distribution from these. - a new config `spark.sql.sources.v2.bucketing.enabled` is introduced to turn on/off the above behavior ### How was this patch tested? - Added a new test suite `KeyGroupedPartitioningSuite` covers end-to-end tests on the new feature - Extended `EnsureRequirementsSuite` to cover `DataSourcePartitioning` - Some existing test classes, such as `InMemoryTable` are extended to cover the changes Closes #35657 from sunchao/SPARK-37377-partitioning. Lead-authored-by: Chao Sun <[email protected]> Co-authored-by: Chao Sun <[email protected]> Signed-off-by: Dongjoon Hyun <[email protected]>
- Loading branch information
1 parent
608f70d
commit 20ffbf7
Showing
45 changed files
with
1,839 additions
and
338 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
44 changes: 0 additions & 44 deletions
44
...catalyst/src/main/java/org/apache/spark/sql/connector/read/partitioning/Distribution.java
This file was deleted.
Oops, something went wrong.
55 changes: 55 additions & 0 deletions
55
...rc/main/java/org/apache/spark/sql/connector/read/partitioning/KeyGroupedPartitioning.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,55 @@ | ||
/* | ||
* 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.connector.read.partitioning; | ||
|
||
import org.apache.spark.annotation.Evolving; | ||
import org.apache.spark.sql.connector.expressions.Expression; | ||
|
||
/** | ||
* Represents a partitioning where rows are split across partitions based on the | ||
* partition transform expressions returned by {@link KeyGroupedPartitioning#keys}. | ||
* <p> | ||
* Note: Data source implementations should make sure for a single partition, all of its rows | ||
* must be evaluated to the same partition value after being applied by | ||
* {@link KeyGroupedPartitioning#keys} expressions. Different partitions can share the same | ||
* partition value: Spark will group these into a single logical partition during planning phase. | ||
* | ||
* @since 3.3.0 | ||
*/ | ||
@Evolving | ||
public class KeyGroupedPartitioning implements Partitioning { | ||
private final Expression[] keys; | ||
private final int numPartitions; | ||
|
||
public KeyGroupedPartitioning(Expression[] keys, int numPartitions) { | ||
this.keys = keys; | ||
this.numPartitions = numPartitions; | ||
} | ||
|
||
/** | ||
* Returns the partition transform expressions for this partitioning. | ||
*/ | ||
public Expression[] keys() { | ||
return keys; | ||
} | ||
|
||
@Override | ||
public int numPartitions() { | ||
return numPartitions; | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
65 changes: 65 additions & 0 deletions
65
sql/catalyst/src/main/scala-2.12/org/apache/spark/sql/catalyst/util/InternalRowSet.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,65 @@ | ||
/* | ||
* 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.catalyst.util | ||
|
||
import scala.collection.mutable | ||
|
||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{Murmur3HashFunction, RowOrdering} | ||
import org.apache.spark.sql.types.{DataType, StructField, StructType} | ||
|
||
/** | ||
* A mutable Set with [[InternalRow]] as its element type. It uses Spark's internal murmur hash to | ||
* compute hash code from an row, and uses [[RowOrdering]] to perform equality checks. | ||
* | ||
* @param dataTypes the data types for the row keys this set holds | ||
*/ | ||
class InternalRowSet(val dataTypes: Seq[DataType]) extends mutable.Set[InternalRow] { | ||
private val baseSet = new mutable.HashSet[InternalRowContainer] | ||
|
||
private val structType = StructType(dataTypes.map(t => StructField("f", t))) | ||
private val ordering = RowOrdering.createNaturalAscendingOrdering(dataTypes) | ||
|
||
override def contains(row: InternalRow): Boolean = | ||
baseSet.contains(new InternalRowContainer(row)) | ||
|
||
private class InternalRowContainer(val row: InternalRow) { | ||
override def hashCode(): Int = Murmur3HashFunction.hash(row, structType, 42L).toInt | ||
|
||
override def equals(other: Any): Boolean = other match { | ||
case r: InternalRowContainer => ordering.compare(row, r.row) == 0 | ||
case r => this == r | ||
} | ||
} | ||
|
||
override def +=(row: InternalRow): InternalRowSet.this.type = { | ||
val rowKey = new InternalRowContainer(row) | ||
baseSet += rowKey | ||
this | ||
} | ||
|
||
override def -=(row: InternalRow): InternalRowSet.this.type = { | ||
val rowKey = new InternalRowContainer(row) | ||
baseSet -= rowKey | ||
this | ||
} | ||
|
||
override def iterator: Iterator[InternalRow] = { | ||
baseSet.iterator.map(_.row) | ||
} | ||
} |
69 changes: 69 additions & 0 deletions
69
sql/catalyst/src/main/scala-2.13/org/apache/spark/sql/catalyst/util/InternalRowSet.scala
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,69 @@ | ||
/* | ||
* 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.catalyst.util | ||
|
||
import scala.collection.mutable | ||
|
||
import org.apache.spark.sql.catalyst.InternalRow | ||
import org.apache.spark.sql.catalyst.expressions.{Murmur3HashFunction, RowOrdering} | ||
import org.apache.spark.sql.types.{DataType, StructField, StructType} | ||
|
||
/** | ||
* A mutable Set with [[InternalRow]] as its element type. It uses Spark's internal murmur hash to | ||
* compute hash code from an row, and uses [[RowOrdering]] to perform equality checks. | ||
* | ||
* @param dataTypes the data types for the row keys this set holds | ||
*/ | ||
class InternalRowSet(val dataTypes: Seq[DataType]) extends mutable.Set[InternalRow] { | ||
private val baseSet = new mutable.HashSet[InternalRowContainer] | ||
|
||
private val structType = StructType(dataTypes.map(t => StructField("f", t))) | ||
private val ordering = RowOrdering.createNaturalAscendingOrdering(dataTypes) | ||
|
||
override def contains(row: InternalRow): Boolean = | ||
baseSet.contains(new InternalRowContainer(row)) | ||
|
||
private class InternalRowContainer(val row: InternalRow) { | ||
override def hashCode(): Int = Murmur3HashFunction.hash(row, structType, 42L).toInt | ||
|
||
override def equals(other: Any): Boolean = other match { | ||
case r: InternalRowContainer => ordering.compare(row, r.row) == 0 | ||
case r => this == r | ||
} | ||
} | ||
|
||
override def addOne(row: InternalRow): InternalRowSet.this.type = { | ||
val rowKey = new InternalRowContainer(row) | ||
baseSet += rowKey | ||
this | ||
} | ||
|
||
override def subtractOne(row: InternalRow): InternalRowSet.this.type = { | ||
val rowKey = new InternalRowContainer(row) | ||
baseSet -= rowKey | ||
this | ||
} | ||
|
||
override def clear(): Unit = { | ||
baseSet.clear() | ||
} | ||
|
||
override def iterator: Iterator[InternalRow] = { | ||
baseSet.iterator.map(_.row) | ||
} | ||
} |
Oops, something went wrong.