rdblue commented on code in PR #4831: URL: https://github.com/apache/iceberg/pull/4831#discussion_r883173588
########## parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java: ########## @@ -0,0 +1,330 @@ +/* + * 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.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.ColumnDescriptor; +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; + + 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); + } + + /** + * 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> fieldsWithNoBloom = null; + private Map<Integer, ColumnDescriptor> cols = null; + private Map<Integer, ColumnChunkMetaData> columnMetaMap = null; + private Map<Integer, BloomFilter> bloomCache = null; + private Map<Integer, Type> types = null; + private boolean hasBloom = false; + + private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) { + this.bloomReader = bloomFilterReader; + this.fieldsWithNoBloom = Sets.newHashSet(); + this.cols = Maps.newHashMap(); + this.columnMetaMap = Maps.newHashMap(); + this.types = Maps.newHashMap(); + + for (ColumnDescriptor desc : fileSchema.getColumns()) { + PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType(); + if (colType.getId() != null) { + int id = colType.getId().intValue(); + cols.put(id, desc); + } + } + + 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); + boolean noBloom = ParquetUtil.hasNoBloomFilterPages(meta); + if (!noBloom) { + hasBloom = true; + } else { + fieldsWithNoBloom.add(id); + } + columnMetaMap.put(id, meta); + types.put(id, icebergType); + } + } + if (!hasBloom) { + return ROWS_MIGHT_MATCH; + } + return ExpressionVisitors.visitEvaluator(expr, this); + } + + @Override + public Boolean alwaysTrue() { + return ROWS_MIGHT_MATCH; // all rows match + } + + @Override + public Boolean alwaysFalse() { + return ROWS_CANNOT_MATCH; // all rows fail + } + + @Override + public Boolean not(Boolean result) { + // not() should be rewritten by RewriteNot + // bloom filter is based on hash and cannot eliminate based on not + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean and(Boolean leftResult, Boolean rightResult) { + return leftResult && rightResult; + } + + @Override + public Boolean or(Boolean leftResult, Boolean rightResult) { + return leftResult || rightResult; + } + + @Override + public <T> Boolean isNull(BoundReference<T> ref) { + // bloom filter only contain non-nulls and cannot eliminate based on isNull or NotNull + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean notNull(BoundReference<T> ref) { + // bloom filter only contain non-nulls and cannot eliminate based on isNull or NotNull + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean isNaN(BoundReference<T> ref) { + // bloom filter is based on hash and cannot eliminate based on isNaN or notNaN + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean notNaN(BoundReference<T> ref) { + // bloom filter is based on hash and cannot eliminate based on isNaN or notNaN + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean lt(BoundReference<T> ref, Literal<T> lit) { + // bloom filter is based on hash and cannot eliminate based on lt or ltEq or gt or gtEq + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean ltEq(BoundReference<T> ref, Literal<T> lit) { + // bloom filter is based on hash and cannot eliminate based on lt or ltEq or gt or gtEq + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean gt(BoundReference<T> ref, Literal<T> lit) { + // bloom filter is based on hash and cannot eliminate based on lt or ltEq or gt or gtEq + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean gtEq(BoundReference<T> ref, Literal<T> lit) { + // bloom filter is based on hash and cannot eliminate based on lt or ltEq or gt or gtEq + return ROWS_MIGHT_MATCH; + } + + @Override + public <T> Boolean eq(BoundReference<T> ref, Literal<T> lit) { + int id = ref.fieldId(); + ColumnDescriptor col = cols.get(id); + if (col == null) { + return ROWS_MIGHT_MATCH; + } else { + Boolean hasNoBloomFilter = fieldsWithNoBloom.contains(id); + if (hasNoBloomFilter) { + return ROWS_MIGHT_MATCH; + } + BloomFilter bloom = getBloomById(id); + Type type = types.get(id); + T value = lit.value(); + return shouldRead(col, value, bloom, type); + } + } + + private BloomFilter getBloomById(int id) { + if (bloomCache == null) { + bloomCache = Maps.newHashMap(); + } + if (bloomCache.containsKey(id)) { + return bloomCache.get(id); + } else { + ColumnChunkMetaData columnChunkMetaData = columnMetaMap.get(id); + BloomFilter bloomFilter = bloomReader.readBloomFilter(columnChunkMetaData); + if (bloomFilter == null) { + throw new IllegalStateException("Failed to read required bloom filter for id: " + id); + } else { + bloomCache.put(id, bloomFilter); + } + + return bloomFilter; + } + } + + private <T> boolean shouldRead(ColumnDescriptor col, T value, BloomFilter bloom, Type type) { Review Comment: Can you move this utility method to the bottom of the class, rather than between override method implementations? -- 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]
