Skip to content

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
wants to merge 3 commits into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,10 @@

import java.io.Serializable;
import java.util.ArrayList;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.avro.Schema;
import org.apache.avro.Schema.Field;
import org.apache.avro.generic.GenericRecord;
Expand All @@ -31,14 +33,17 @@
import org.apache.beam.sdk.extensions.sql.meta.SchemaBaseBeamTable;
import org.apache.beam.sdk.extensions.sql.meta.Table;
import org.apache.beam.sdk.io.FileIO;
import org.apache.beam.sdk.io.parquet.ParquetFilterFactory;
import org.apache.beam.sdk.io.parquet.ParquetIO;
import org.apache.beam.sdk.io.parquet.ParquetIO.Read;
import org.apache.beam.sdk.schemas.transforms.Convert;
import org.apache.beam.sdk.schemas.transforms.Select;
import org.apache.beam.sdk.values.PBegin;
import org.apache.beam.sdk.values.PCollection;
import org.apache.beam.sdk.values.PCollection.IsBounded;
import org.apache.beam.sdk.values.POutput;
import org.apache.beam.sdk.values.Row;
import org.apache.beam.vendor.calcite.v1_28_0.org.apache.calcite.rex.RexNode;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -64,23 +69,70 @@ public PCollection<Row> buildIOReader(PBegin begin) {

@Override
public PCollection<Row> buildIOReader(
PBegin begin, BeamSqlTableFilter filters, List<String> fieldNames) {
PBegin begin, BeamSqlTableFilter filter, List<String> projectedFieldNames) {

// Determine ALL fields required for the read (projection + filter fields).
Set<String> requiredFieldsForRead = new HashSet<>(projectedFieldNames);
if (filter instanceof ParquetFilter) {
ParquetFilter parquetFilter = (ParquetFilter) filter;
requiredFieldsForRead.addAll(parquetFilter.getReferencedFields(getSchema()));
}

// If no fields are projected or filtered, read the full schema.
final Schema schema = AvroUtils.toAvroSchema(table.getSchema());
Schema readSchema =
requiredFieldsForRead.isEmpty()
? schema
: projectSchema(schema, new ArrayList<>(requiredFieldsForRead));

LOG.info("Projecting fields schema: {}", readSchema);

String filePattern = resolveFilePattern(table.getLocation());
Read read = ParquetIO.read(schema).withBeamSchemas(true).from(filePattern);
if (!fieldNames.isEmpty()) {
Schema projectionSchema = projectSchema(schema, fieldNames);
LOG.info("Projecting fields schema: {}", projectionSchema);
read = read.withProjection(projectionSchema, projectionSchema);
read = read.withProjection(readSchema, readSchema);
Copy link
Contributor

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:

* <p>Reading with projection can be enabled with the projection schema as following. Splittable
* reading is enabled when reading with projection. The projection_schema contains only the column
* that we would like to read and encoder_schema contains the schema to encode the output with the
* unwanted columns changed to nullable. Partial reading provide decrease of reading time due to
* partial processing of the data and partial encoding. The decrease in the reading time depends on


if (filter instanceof ParquetFilter) {
ParquetFilter parquetFilter = (ParquetFilter) filter;
List<RexNode> supported = parquetFilter.getSupported();
if (!supported.isEmpty()) {
org.apache.beam.sdk.io.parquet.ParquetFilter predicate =
ParquetFilterFactory.create(supported, getSchema());
read = read.withFilter(predicate);
}
}
return begin.apply("ParquetIORead", read).apply("ToRows", Convert.toRows());

PCollection<Row> rowsWithRequiredFields =
begin.apply("ParquetIORead With Filtering", read).apply("ToRows", Convert.toRows());

// If we read extra fields for filtering, project them away to match the final SELECT list.
if (!projectedFieldNames.isEmpty()
&& !new HashSet<>(projectedFieldNames).equals(requiredFieldsForRead)) {
return rowsWithRequiredFields.apply(
Select.fieldNames(projectedFieldNames.toArray(new String[0])));
} else {
return rowsWithRequiredFields;
}
}

@Override
public BeamSqlTableFilter constructFilter(List<RexNode> filter) {
return new ParquetFilter(filter);
}

/** Returns a copy of the {@link Schema} with only the fieldNames fields. */
private static Schema projectSchema(Schema schema, List<String> fieldNames) {
if (fieldNames.isEmpty()) {
return schema;
}

Set<String> fieldNameSet = new HashSet<>(fieldNames);
List<Field> selectedFields = new ArrayList<>();
for (String fieldName : fieldNames) {
selectedFields.add(deepCopyField(schema.getField(fieldName)));

// Iterate through the original schema's fields to maintain their order.
for (Schema.Field field : schema.getFields()) {
if (fieldNameSet.contains(field.name())) {
selectedFields.add(deepCopyField(field));
}
}
return Schema.createRecord(
schema.getName() + "_projected",
Expand Down
Loading
Loading