-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-40259][SQL] Support Parquet DSv2 in subquery plan merge #37711
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
Changes from all 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 |
|---|---|---|
| @@ -0,0 +1,38 @@ | ||
| /* | ||
| * 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; | ||
|
|
||
| import java.util.Optional; | ||
|
|
||
| import org.apache.spark.annotation.Evolving; | ||
| import org.apache.spark.sql.connector.catalog.SupportsRead; | ||
|
|
||
| /** | ||
| * A mix in interface for {@link Scan}. Data sources can implement this interface to indicate | ||
| * {@link Scan}s can be merged. | ||
| * | ||
| * @since 3.4.0 | ||
| */ | ||
| @Evolving | ||
| public interface SupportsMerge extends Scan { | ||
|
|
||
| /** | ||
| * Returns the merged scan. | ||
| */ | ||
| Optional<SupportsMerge> mergeWith(SupportsMerge other, SupportsRead table); | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -16,6 +16,8 @@ | |
| */ | ||
| package org.apache.spark.sql.execution.datasources.v2.parquet | ||
|
|
||
| import java.util.Optional | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.hadoop.conf.Configuration | ||
|
|
@@ -24,8 +26,9 @@ import org.apache.parquet.hadoop.ParquetInputFormat | |
|
|
||
| import org.apache.spark.sql.SparkSession | ||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.connector.catalog.SupportsRead | ||
| import org.apache.spark.sql.connector.expressions.aggregate.Aggregation | ||
| import org.apache.spark.sql.connector.read.PartitionReaderFactory | ||
| import org.apache.spark.sql.connector.read.{PartitionReaderFactory, SupportsMerge} | ||
| import org.apache.spark.sql.execution.datasources.{AggregatePushDownUtils, PartitioningAwareFileIndex, RowIndexUtil} | ||
| import org.apache.spark.sql.execution.datasources.parquet.{ParquetOptions, ParquetReadSupport, ParquetWriteSupport} | ||
| import org.apache.spark.sql.execution.datasources.v2.FileScan | ||
|
|
@@ -46,7 +49,7 @@ case class ParquetScan( | |
| options: CaseInsensitiveStringMap, | ||
| pushedAggregate: Option[Aggregation] = None, | ||
| partitionFilters: Seq[Expression] = Seq.empty, | ||
| dataFilters: Seq[Expression] = Seq.empty) extends FileScan { | ||
| dataFilters: Seq[Expression] = Seq.empty) extends FileScan with SupportsMerge { | ||
| override def isSplitable(path: Path): Boolean = { | ||
| // If aggregate is pushed down, only the file footer will be read once, | ||
| // so file should not be split across multiple tasks. | ||
|
|
@@ -106,15 +109,18 @@ case class ParquetScan( | |
| new ParquetOptions(options.asCaseSensitiveMap.asScala.toMap, sqlConf)) | ||
| } | ||
|
|
||
| private def pushedDownAggEqual(p: ParquetScan) = { | ||
| if (pushedAggregate.nonEmpty && p.pushedAggregate.nonEmpty) { | ||
| AggregatePushDownUtils.equivalentAggregations(pushedAggregate.get, p.pushedAggregate.get) | ||
| } else { | ||
| pushedAggregate.isEmpty && p.pushedAggregate.isEmpty | ||
| } | ||
| } | ||
|
Comment on lines
+112
to
+118
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. should we move this to FileScan itself ? OrcScan also has some duplicate code |
||
|
|
||
| override def equals(obj: Any): Boolean = obj match { | ||
| case p: ParquetScan => | ||
| val pushedDownAggEqual = if (pushedAggregate.nonEmpty && p.pushedAggregate.nonEmpty) { | ||
| AggregatePushDownUtils.equivalentAggregations(pushedAggregate.get, p.pushedAggregate.get) | ||
| } else { | ||
| pushedAggregate.isEmpty && p.pushedAggregate.isEmpty | ||
| } | ||
| super.equals(p) && dataSchema == p.dataSchema && options == p.options && | ||
| equivalentFilters(pushedFilters, p.pushedFilters) && pushedDownAggEqual | ||
| equivalentFilters(pushedFilters, p.pushedFilters) && pushedDownAggEqual(p) | ||
| case _ => false | ||
| } | ||
|
|
||
|
|
@@ -138,4 +144,29 @@ case class ParquetScan( | |
| Map("PushedAggregation" -> pushedAggregationsStr) ++ | ||
| Map("PushedGroupBy" -> pushedGroupByStr) | ||
| } | ||
|
|
||
| override def mergeWith(other: SupportsMerge, table: SupportsRead): Optional[SupportsMerge] = { | ||
| if (other.isInstanceOf[ParquetScan]) { | ||
|
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. can replace this with case match |
||
| val o = other.asInstanceOf[ParquetScan] | ||
| if (fileIndex == o.fileIndex && | ||
| options == o.options && | ||
| dataSchema == o.dataSchema && | ||
| equivalentFilters(pushedFilters, o.pushedFilters) && | ||
| pushedDownAggEqual(o) && | ||
| normalizedPartitionFilters == o.normalizedPartitionFilters && | ||
| normalizedDataFilters == o.normalizedDataFilters) { | ||
|
Comment on lines
+156
to
+157
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. [question] should we just disjunct these diff filters from scans and run a boolean simplification on top of it ? to handle the cases with diff partition and data filter on the scans ? Are we expecting some heuristic here ? as if when combining the filters will be useful ? |
||
| val builder = table.newScanBuilder(options).asInstanceOf[ParquetScanBuilder] | ||
|
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. [question] should we add assertion for |
||
| pushedAggregate.map(builder.pushAggregation) | ||
| builder.pushFilters(dataFilters ++ partitionFilters) | ||
| builder.pruneColumns(readSchema().merge(o.readSchema())) | ||
| val scan = builder.build().asInstanceOf[ParquetScan] | ||
|
|
||
| Optional.of(scan) | ||
| } else { | ||
| Optional.empty() | ||
| } | ||
| } else { | ||
| Optional.empty() | ||
| } | ||
| } | ||
| } | ||
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.
[minor] can we simplify the if else structure here ? something like