rdblue commented on code in PR #4831: URL: https://github.com/apache/iceberg/pull/4831#discussion_r884960868
########## parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java: ########## @@ -0,0 +1,316 @@ +/* + * 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.iceberg.parquet; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.BoundReference; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; +import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.iceberg.util.UUIDUtil; +import org.apache.parquet.column.values.bloomfilter.BloomFilter; +import org.apache.parquet.hadoop.BloomFilterReader; +import org.apache.parquet.hadoop.metadata.BlockMetaData; +import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.DecimalMetadata; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; + +public class ParquetBloomRowGroupFilter { + private final Schema schema; + private final Expression expr; + private final boolean caseSensitive; + + public ParquetBloomRowGroupFilter(Schema schema, Expression unbound) { + this(schema, unbound, true); + } + + public ParquetBloomRowGroupFilter(Schema schema, Expression unbound, boolean caseSensitive) { + this.schema = schema; + StructType struct = schema.asStruct(); + this.expr = Binder.bind(struct, Expressions.rewriteNot(unbound), caseSensitive); + this.caseSensitive = caseSensitive; + } + + /** + * Tests whether the bloom for a row group may contain records that match the expression. + * + * @param fileSchema schema for the Parquet file + * @param bloomReader a bloom filter reader + * @return false if the file cannot contain rows that match the expression, true otherwise. + */ + public boolean shouldRead(MessageType fileSchema, BlockMetaData rowGroup, + BloomFilterReader bloomReader) { + return new BloomEvalVisitor().eval(fileSchema, rowGroup, bloomReader); + } + + private static final boolean ROWS_MIGHT_MATCH = true; + private static final boolean ROWS_CANNOT_MATCH = false; + + private class BloomEvalVisitor extends BoundExpressionVisitor<Boolean> { + private BloomFilterReader bloomReader; + private Set<Integer> fieldsWithBloomFilter = null; + private Map<Integer, ColumnChunkMetaData> columnMetaMap = null; + private Map<Integer, BloomFilter> bloomCache = null; + private Map<Integer, PrimitiveType> parquetPrimitiveTypes = null; + private Map<Integer, Type> types = null; + + private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) { + this.bloomReader = bloomFilterReader; + this.fieldsWithBloomFilter = Sets.newHashSet(); + this.columnMetaMap = Maps.newHashMap(); + this.bloomCache = Maps.newHashMap(); + this.parquetPrimitiveTypes = Maps.newHashMap(); + this.types = Maps.newHashMap(); + + for (ColumnChunkMetaData meta : rowGroup.getColumns()) { + PrimitiveType colType = fileSchema.getType(meta.getPath().toArray()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + Type icebergType = schema.findType(id); + if (!ParquetUtil.hasNoBloomFilterPages(meta)) { + fieldsWithBloomFilter.add(id); + } + columnMetaMap.put(id, meta); + parquetPrimitiveTypes.put(id, colType); + types.put(id, icebergType); + } + } + + Set<Integer> filterRefs = Binder.boundReferences(schema.asStruct(), ImmutableList.of(expr), caseSensitive, true); + // If the filter's column set doesn't overlap with any bloom filter columns, exit early with ROWS_MIGHT_MATCH + if (filterRefs.size() > 0 && Sets.intersection(fieldsWithBloomFilter, filterRefs).isEmpty()) { + return ROWS_MIGHT_MATCH; + } + return ExpressionVisitors.visitEvaluator(expr, this); Review Comment: Style: there should be a newline between a control flow block and the following statement. -- 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]
