-
Notifications
You must be signed in to change notification settings - Fork 4.4k
Added Filter and Projection Pushdown support to ParquetIO and Beam SQL's ParquetTable #35589
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
Open
talatuyarer
wants to merge
3
commits into
apache:master
Choose a base branch
from
talatuyarer:parquet-filter-pushdown-support
base: master
Could not load branches
Branch not found: {{ refName }}
Loading
Could not load tags
Nothing to show
Loading
Are you sure you want to change the base?
Some commits from the old base branch may be removed from the timeline,
and old review comments may become outdated.
Open
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
141 changes: 141 additions & 0 deletions
141
...src/main/java/org/apache/beam/sdk/extensions/sql/meta/provider/parquet/ParquetFilter.java
This file contains hidden or 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,141 @@ | ||
/* | ||
* 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.beam.sdk.extensions.sql.meta.provider.parquet; | ||
|
||
import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; | ||
|
||
import java.util.HashSet; | ||
import java.util.List; | ||
import java.util.Set; | ||
import org.apache.beam.sdk.extensions.sql.meta.BeamSqlTableFilter; | ||
import org.apache.beam.sdk.schemas.Schema; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexCall; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexInputRef; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexLiteral; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.sql.SqlKind; | ||
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.commons.lang3.tuple.Pair; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; | ||
import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; | ||
|
||
/** | ||
* A {@link BeamSqlTableFilter} for ParquetIO that classifies filters as supported or unsupported. | ||
*/ | ||
public class ParquetFilter implements BeamSqlTableFilter { | ||
// The set of operators that can be pushed down. | ||
private static final ImmutableSet<SqlKind> SUPPORTED_OPS = | ||
ImmutableSet.of( | ||
SqlKind.AND, | ||
SqlKind.OR, | ||
SqlKind.NOT, | ||
SqlKind.EQUALS, | ||
SqlKind.NOT_EQUALS, | ||
SqlKind.GREATER_THAN, | ||
SqlKind.GREATER_THAN_OR_EQUAL, | ||
SqlKind.LESS_THAN, | ||
SqlKind.LESS_THAN_OR_EQUAL, | ||
SqlKind.IN, | ||
SqlKind.IS_NULL, | ||
SqlKind.IS_NOT_NULL); | ||
|
||
private final List<RexNode> supported; | ||
private final List<RexNode> unsupported; | ||
|
||
public ParquetFilter(List<RexNode> predicateCNF) { | ||
Pair<List<RexNode>, List<RexNode>> classifiedFilters = classify(predicateCNF); | ||
this.supported = classifiedFilters.getLeft(); | ||
this.unsupported = classifiedFilters.getRight(); | ||
} | ||
|
||
public Set<String> getReferencedFields(Schema beamSchema) { | ||
Set<String> fields = new HashSet<>(); | ||
for (RexNode node : supported) { | ||
collectReferencedFields(node, beamSchema, fields); | ||
} | ||
return fields; | ||
} | ||
|
||
private static void collectReferencedFields(RexNode node, Schema beamSchema, Set<String> fields) { | ||
if (node instanceof RexInputRef) { | ||
fields.add(beamSchema.getField(((RexInputRef) node).getIndex()).getName()); | ||
} else if (node instanceof RexCall) { | ||
for (RexNode operand : ((RexCall) node).getOperands()) { | ||
collectReferencedFields(operand, beamSchema, fields); | ||
} | ||
} | ||
} | ||
|
||
/** Static helper method to classify filters. */ | ||
private static Pair<List<RexNode>, List<RexNode>> classify(List<RexNode> predicates) { | ||
ImmutableList.Builder<RexNode> supportedBuilder = ImmutableList.builder(); | ||
ImmutableList.Builder<RexNode> unsupportedBuilder = ImmutableList.builder(); | ||
|
||
for (RexNode node : predicates) { | ||
if (isSupported(node).getLeft()) { | ||
supportedBuilder.add(node); | ||
} else { | ||
unsupportedBuilder.add(node); | ||
} | ||
} | ||
return Pair.of(supportedBuilder.build(), unsupportedBuilder.build()); | ||
} | ||
|
||
@Override | ||
public List<RexNode> getNotSupported() { | ||
return unsupported; | ||
} | ||
|
||
@Override | ||
public int numSupported() { | ||
return BeamSqlTableFilter.expressionsInFilter(checkStateNotNull(supported)); | ||
} | ||
|
||
public List<RexNode> getSupported() { | ||
return supported; | ||
} | ||
|
||
private static Pair<Boolean, Integer> isSupported(RexNode node) { | ||
if (!(node instanceof RexCall)) { | ||
return Pair.of(node instanceof RexLiteral || node instanceof RexInputRef, 0); | ||
} | ||
|
||
RexCall call = (RexCall) node; | ||
if (!SUPPORTED_OPS.contains(call.getKind())) { | ||
return Pair.of(false, 0); | ||
} | ||
|
||
boolean allOperandsSupported = true; | ||
int inputRefCount = 0; | ||
for (RexNode operand : call.getOperands()) { | ||
if (operand instanceof RexInputRef) { | ||
inputRefCount++; | ||
} else if (operand instanceof RexCall) { | ||
Pair<Boolean, Integer> childSupport = isSupported(operand); | ||
if (!childSupport.getLeft()) { | ||
allOperandsSupported = false; | ||
break; | ||
} | ||
inputRefCount += childSupport.getRight(); | ||
} | ||
} | ||
|
||
boolean isStructureSupported = inputRefCount <= 1; | ||
|
||
return Pair.of(allOperandsSupported && isStructureSupported, inputRefCount); | ||
} | ||
} |
This file contains hidden or 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
Oops, something went wrong.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back 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.
I think we should be making the unwanted columns nullable for the second
readSchema
. Check ParquetIO javadoc:beam/sdks/java/io/parquet/src/main/java/org/apache/beam/sdk/io/parquet/ParquetIO.java
Lines 137 to 141 in cd72a25