You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2021/05/11 12:45:49 UTC

[GitHub] [iceberg] jshmchenxi opened a new pull request #2582: Add support for Parquet BloomFilter

jshmchenxi opened a new pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582


   For #2391, add Parquet BloomFilter support to Iceberg.
   Upgrade Parquet version to 1.12.0 and add ParquetBloomRowGroupFilter similar to ParquetDictionaryRowGroupFilter.
   
   ExpressionVisitor is implemented with refer to [org.apache.parquet.filter2.bloomfilterlevel.BloomFilterImpl](https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-hadoop/src/main/java/org/apache/parquet/filter2/bloomfilterlevel/BloomFilterImpl.java).
   BloomFilter is helpful only with eq() and in() expression. It can not help filtering rows with other expressions like gt() or notEq().
   
   Add 3 new properties to TableProperties. The definition is similar to #parquet-mr
   
   - write.parquet.bloom-filter-enabled
   - write.parquet.bloom-filter-max-bytes
   - write.parquet.bloom-filter-expected-ndv


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r630145366



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Type;
+import org.apache.iceberg.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = Maps.newHashMap();
+
+      for (ColumnDescriptor desc : fileSchema.getColumns()) {
+        PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType();
+        if (colType.getId() != null) {
+          int id = colType.getId().intValue();
+          Type icebergType = schema.findType(id);
+          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();
+          isFallback.put(id, ParquetUtil.hasNonBloomFilterPages(meta));
+          columnMetaMap.put(id, meta);
+        }
+      }
+
+      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) {
+      throw new IllegalArgumentException(
+          "Bloom filter does not support not expression!");
+    }
+
+    @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();
+
+      Boolean hasNonBloomFilter = isFallback.get(id);
+      if (hasNonBloomFilter == null || hasNonBloomFilter) {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      BloomFilter bloom = getBloomById(id);
+      T value = lit.value();
+      return bloom.findHash(tryHash(id, value, bloom)) ? ROWS_MIGHT_MATCH : ROWS_CANNOT_MATCH;
+    }
+
+    @Override
+    public <T> Boolean notEq(BoundReference<T> ref, Literal<T> lit) {
+      // bloom filter is based on hash and cannot eliminate based on notEq
+      return ROWS_MIGHT_MATCH;
+    }
+
+    @Override
+    public <T> Boolean in(BoundReference<T> ref, Set<T> literalSet) {
+      int id = ref.fieldId();
+
+      Boolean hasNonBloomFilter = isFallback.get(id);
+      if (hasNonBloomFilter == null || hasNonBloomFilter) {
+        return ROWS_MIGHT_MATCH;
+      }
+
+      BloomFilter bloom = getBloomById(id);
+      for (T e : literalSet) {
+        if (bloom.findHash(tryHash(id, e, bloom))) {
+          // found hash so rows match
+          return ROWS_MIGHT_MATCH;
+        }
+      }
+
+      // no hash found so rows don't match
+      return ROWS_CANNOT_MATCH;
+    }
+
+    @Override
+    public <T> Boolean notIn(BoundReference<T> ref, Set<T> literalSet) {
+      // bloom filter is based on hash and cannot eliminate based on notIn
+      return ROWS_MIGHT_MATCH;
+    }
+
+    @Override
+    public <T> Boolean startsWith(BoundReference<T> ref, Literal<T> lit) {
+      // bloom filter is based on hash and cannot eliminate based on startsWith
+      return ROWS_MIGHT_MATCH;
+    }
+
+    private BloomFilter getBloomById(int id) {
+      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);
+      }
+
+      return bloomFilter;
+    }
+
+    private <T> long tryHash(int id, T value, BloomFilter bloom) {
+      ColumnDescriptor col = cols.get(id);
+
+      switch (col.getPrimitiveType().getPrimitiveTypeName()) {
+        case BINARY:
+          return bloom.hash(Binary.fromString(value.toString()));

Review comment:
       In case BINARY, we ran into "Not supported type" error with [BlockSplitBloomFilter.hash()](https://github.com/apache/parquet-mr/blob/apache-parquet-1.12.0/parquet-column/src/main/java/org/apache/parquet/column/values/bloomfilter/BlockSplitBloomFilter.java#L312) because String is not of type Binary.
   This is an ugly solution and it only considers String type under BINARY.
   Maybe there needs to be a function to transfer Iceberg type to Parquet type, like the reverse of ParquetConversions.converterFromParquet(colType, icebergType)?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r632275827



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),
+      optional(18, "_no_stats", StringType.get())
+  );
+
+  private static final String TOO_LONG_FOR_STATS;
+
+  static {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 200; i += 1) {
+      sb.append(UUID.randomUUID().toString());
+    }
+    TOO_LONG_FOR_STATS = sb.toString();
+  }
+
+  private static final int INT_MIN_VALUE = 30;
+  private static final int INT_MAX_VALUE = 79;
+  private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1;
+  private static final long LONG_BASE = 100L;
+  private static final double DOUBLE_BASE = 1000D;
+  private static final float FLOAT_BASE = 10000F;
+  private static final String BINARY_PREFIX = "BINARY测试_";
+
+  private static final List<String> RANDOM_UUIDS;
+
+  static {
+    RANDOM_UUIDS = new ArrayList<>();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID().toString());
+    }
+  }
+
+  private MessageType parquetSchema = null;
+  private BlockMetaData rowGroupMetadata = null;
+  private BloomFilterReader bloomStore = null;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createInputFile() throws IOException {
+    File parquetFile = temp.newFile();
+    Assert.assertTrue(parquetFile.delete());
+
+    // build struct field schema
+    org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType);
+
+    OutputFile outFile = Files.localOutput(parquetFile);
+    try (FileAppender<Record> appender = Parquet.write(outFile)
+        .schema(FILE_SCHEMA)
+        .set(PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_ENABLED + "#_non_bloom", "false")

Review comment:
       This is used to disable bloom filter for the specified column `_non_bloom`.
   The definition of `write.parquet.bloom-filter-enabled` is similar to `parquet.bloom.filter.enabled` in [apache/parquet-mr](https://github.com/apache/parquet-mr/tree/master/parquet-hadoop). 
   
   > It is also possible to enable it for some columns by specifying the column name within the property followed by #.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r650448248



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ColumnConfigParser.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * TODO: Once org.apache.parquet.hadoop.ColumnConfigParser is made public, should replace this class.
+ * Parses the specified key-values in the format of root.key#column.path from a {@link Configuration} object.
+ */
+class ColumnConfigParser {

Review comment:
       Iceberg doesn't use the same names that Parquet would, and it also doesn't use a `Configuration` to store properties. We need to think about what would make sense for Iceberg here, and using `#` to delimit properties is probably too confusing.
   
   I think that the properties proposed in this PR for global defaults make sense, like `write.parquet.bloom-filter-enabled`, although the NDV default is probably not useful given that we expect NDV to vary widely across fields. For the column-specific settings, I think we may want to follow the same pattern that is used by metrics collection. That embeds the column name in the property, like `write.metadata.metrics.column.col1`. This could be `write.parquet.bloom-filter.col1.enabled` or `write.parquet.bloom-filter.col1.max-bytes`.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = 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();
+          isFallback.put(id, ParquetUtil.hasNonBloomFilterPages(meta));
+          columnMetaMap.put(id, meta);
+        }
+      }
+
+      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) {
+      throw new IllegalArgumentException(
+          "Bloom filter does not support not expression!");
+    }

Review comment:
       I think it's fine to throw an exception here because it indicates that there is a problem and `RewriteNot` was not called. The error class and message should indicate that it is a bug. I'd probably throw `UnsupportedOperationException` with an error message that states the expression violated assumptions and there is probably a bug because `RewriteNot` was not called.

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java
##########
@@ -195,11 +196,12 @@ private void startRowGroup() {
     this.nextCheckRecordCount = Math.min(Math.max(recordCount / 2, 100), 10000);
     this.recordCount = 0;
 
-    PageWriteStore pageStore = pageStoreCtorParquet.newInstance(
+    ColumnChunkPageWriteStore pageStore = pageStoreCtorParquet.newInstance(

Review comment:
       Why are there write-side changes in this PR?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
kbendick commented on pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#issuecomment-846325291


   > @jshmchenxi, I think this should be done in several PRs instead of one. First, we would need to update the Parquet version, then we would want to add read support and finally we would add write support. That will help keep the changes to a size where reviewers can get through them in a reasonable amount of time
   
   Agreed on parquet versions. With the number of supported spark versions, it would be difficult to bring up parquet 1.12 (as great as it is) without some consideration by major stakeholders.
   
   


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r667331174



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),
+      optional(18, "_no_stats", StringType.get())
+  );
+
+  private static final String TOO_LONG_FOR_STATS;
+
+  static {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 200; i += 1) {
+      sb.append(UUID.randomUUID().toString());
+    }
+    TOO_LONG_FOR_STATS = sb.toString();
+  }
+
+  private static final int INT_MIN_VALUE = 30;
+  private static final int INT_MAX_VALUE = 79;
+  private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1;
+  private static final long LONG_BASE = 100L;
+  private static final double DOUBLE_BASE = 1000D;
+  private static final float FLOAT_BASE = 10000F;
+  private static final String BINARY_PREFIX = "BINARY测试_";
+
+  private static final List<String> RANDOM_UUIDS;
+
+  static {
+    RANDOM_UUIDS = new ArrayList<>();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID().toString());
+    }
+  }
+
+  private MessageType parquetSchema = null;
+  private BlockMetaData rowGroupMetadata = null;
+  private BloomFilterReader bloomStore = null;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createInputFile() throws IOException {
+    File parquetFile = temp.newFile();
+    Assert.assertTrue(parquetFile.delete());
+
+    // build struct field schema
+    org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType);
+
+    OutputFile outFile = Files.localOutput(parquetFile);
+    try (FileAppender<Record> appender = Parquet.write(outFile)
+        .schema(FILE_SCHEMA)
+        .set(PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_ENABLED + "#_non_bloom", "false")
+        .build()) {
+      GenericRecordBuilder builder = new GenericRecordBuilder(convert(FILE_SCHEMA, "table"));
+      // create 50 records
+      for (int i = 0; i < INT_VALUE_COUNT; i += 1) {

Review comment:
       @openinx Thanks! That's a good idea. I'll try using the `RandomGenericData` utility to generate test cases.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] dixingxing0 commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
dixingxing0 commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r630144256



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Type;
+import org.apache.iceberg.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = Maps.newHashMap();
+
+      for (ColumnDescriptor desc : fileSchema.getColumns()) {
+        PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType();
+        if (colType.getId() != null) {
+          int id = colType.getId().intValue();
+          Type icebergType = schema.findType(id);

Review comment:
       curious about here, `iceberType` seems useless, or should we add some comments here?




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r666807237



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),
+      optional(18, "_no_stats", StringType.get())
+  );
+
+  private static final String TOO_LONG_FOR_STATS;
+
+  static {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 200; i += 1) {
+      sb.append(UUID.randomUUID().toString());
+    }
+    TOO_LONG_FOR_STATS = sb.toString();
+  }
+
+  private static final int INT_MIN_VALUE = 30;
+  private static final int INT_MAX_VALUE = 79;
+  private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1;
+  private static final long LONG_BASE = 100L;
+  private static final double DOUBLE_BASE = 1000D;
+  private static final float FLOAT_BASE = 10000F;
+  private static final String BINARY_PREFIX = "BINARY测试_";
+
+  private static final List<String> RANDOM_UUIDS;
+
+  static {
+    RANDOM_UUIDS = new ArrayList<>();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID().toString());
+    }
+  }
+
+  private MessageType parquetSchema = null;
+  private BlockMetaData rowGroupMetadata = null;
+  private BloomFilterReader bloomStore = null;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createInputFile() throws IOException {
+    File parquetFile = temp.newFile();
+    Assert.assertTrue(parquetFile.delete());
+
+    // build struct field schema
+    org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType);
+
+    OutputFile outFile = Files.localOutput(parquetFile);
+    try (FileAppender<Record> appender = Parquet.write(outFile)
+        .schema(FILE_SCHEMA)
+        .set(PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_ENABLED + "#_non_bloom", "false")
+        .build()) {
+      GenericRecordBuilder builder = new GenericRecordBuilder(convert(FILE_SCHEMA, "table"));
+      // create 50 records
+      for (int i = 0; i < INT_VALUE_COUNT; i += 1) {

Review comment:
       We usually use the `org.apache.iceberg.data.RandomGenericData#generate` to generate random `Record` for testing purpose because it could cover almost all the corner cases that will encounter in the real production (Actually,  I detected several bugs when I use the `RandomGenericData` to mock data and run unit tests).  I think we could also use it here. For example, we may generate several records into a collection and then check whether all the values from the given column are shown positive in the parquet bloom filter binary.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] kbendick commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
kbendick commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r632102293



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),

Review comment:
       Can you please add a test for this column using an in or equals predicate to ensure that the existence of a bloom filter on a file for a query against a field that doesn't work for the bloom filter doesn't throw?

##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),
+      optional(18, "_no_stats", StringType.get())
+  );
+
+  private static final String TOO_LONG_FOR_STATS;
+
+  static {
+    StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < 200; i += 1) {
+      sb.append(UUID.randomUUID().toString());
+    }
+    TOO_LONG_FOR_STATS = sb.toString();
+  }
+
+  private static final int INT_MIN_VALUE = 30;
+  private static final int INT_MAX_VALUE = 79;
+  private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1;
+  private static final long LONG_BASE = 100L;
+  private static final double DOUBLE_BASE = 1000D;
+  private static final float FLOAT_BASE = 10000F;
+  private static final String BINARY_PREFIX = "BINARY测试_";
+
+  private static final List<String> RANDOM_UUIDS;
+
+  static {
+    RANDOM_UUIDS = new ArrayList<>();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID().toString());
+    }
+  }
+
+  private MessageType parquetSchema = null;
+  private BlockMetaData rowGroupMetadata = null;
+  private BloomFilterReader bloomStore = null;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void createInputFile() throws IOException {
+    File parquetFile = temp.newFile();
+    Assert.assertTrue(parquetFile.delete());
+
+    // build struct field schema
+    org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType);
+
+    OutputFile outFile = Files.localOutput(parquetFile);
+    try (FileAppender<Record> appender = Parquet.write(outFile)
+        .schema(FILE_SCHEMA)
+        .set(PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_ENABLED + "#_non_bloom", "false")

Review comment:
       What does this config parameter do? Is it something we should include as an option directly?

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = 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();
+          isFallback.put(id, ParquetUtil.hasNonBloomFilterPages(meta));
+          columnMetaMap.put(id, meta);
+        }
+      }
+
+      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) {
+      throw new IllegalArgumentException(
+          "Bloom filter does not support not expression!");
+    }

Review comment:
       Will `RewriteNot` have already been processed by this point in time (and hence this exception shouldn't be reached)? We wouldn't want to throw on any query against a file with a bloom filter that has `not` in it. 




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#issuecomment-839367064


   @chenjunjiedada Hi, would you please help review this patch? Thanks!


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r632276240



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),

Review comment:
       OK. I'll try later. Thanks for the review!




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r651395698



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ColumnConfigParser.java
##########
@@ -0,0 +1,82 @@
+/*
+ * 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.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.function.BiConsumer;
+import java.util.function.Function;
+import org.apache.hadoop.conf.Configuration;
+
+/**
+ * TODO: Once org.apache.parquet.hadoop.ColumnConfigParser is made public, should replace this class.
+ * Parses the specified key-values in the format of root.key#column.path from a {@link Configuration} object.
+ */
+class ColumnConfigParser {

Review comment:
       Ok, I will try to change the configuration pattern when I have time.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r630146734



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,260 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Type;
+import org.apache.iceberg.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = Maps.newHashMap();
+
+      for (ColumnDescriptor desc : fileSchema.getColumns()) {
+        PrimitiveType colType = fileSchema.getType(desc.getPath()).asPrimitiveType();
+        if (colType.getId() != null) {
+          int id = colType.getId().intValue();
+          Type icebergType = schema.findType(id);

Review comment:
       I forgot to delete it. Thanks!




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] openinx commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r666800498



##########
File path: parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java
##########
@@ -0,0 +1,1073 @@
+/*
+ * 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.io.File;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.avro.generic.GenericData.Record;
+import org.apache.avro.generic.GenericRecordBuilder;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestHelpers;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.exceptions.ValidationException;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.types.Types;
+import org.apache.iceberg.types.Types.DoubleType;
+import org.apache.iceberg.types.Types.FloatType;
+import org.apache.iceberg.types.Types.IntegerType;
+import org.apache.iceberg.types.Types.LongType;
+import org.apache.iceberg.types.Types.StringType;
+import org.apache.parquet.column.values.bloomfilter.BloomFilter;
+import org.apache.parquet.hadoop.BloomFilterReader;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.schema.MessageType;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.avro.AvroSchemaUtil.convert;
+import static org.apache.iceberg.expressions.Expressions.and;
+import static org.apache.iceberg.expressions.Expressions.equal;
+import static org.apache.iceberg.expressions.Expressions.greaterThan;
+import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.in;
+import static org.apache.iceberg.expressions.Expressions.isNaN;
+import static org.apache.iceberg.expressions.Expressions.isNull;
+import static org.apache.iceberg.expressions.Expressions.lessThan;
+import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual;
+import static org.apache.iceberg.expressions.Expressions.not;
+import static org.apache.iceberg.expressions.Expressions.notEqual;
+import static org.apache.iceberg.expressions.Expressions.notIn;
+import static org.apache.iceberg.expressions.Expressions.notNaN;
+import static org.apache.iceberg.expressions.Expressions.notNull;
+import static org.apache.iceberg.expressions.Expressions.or;
+import static org.apache.iceberg.expressions.Expressions.startsWith;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestBloomRowGroupFilter {
+
+  private static final Types.StructType structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get()));
+
+  private static final Schema SCHEMA = new Schema(
+      required(1, "id", IntegerType.get()),
+      required(2, "id_long", LongType.get()),
+      required(3, "id_double", DoubleType.get()),
+      required(4, "id_float", FloatType.get()),
+      required(5, "id_binary", StringType.get()),
+      required(6, "random_binary", StringType.get()),
+      required(7, "required", StringType.get()),
+      optional(8, "non_bloom", StringType.get()),
+      optional(9, "all_nulls", LongType.get()),
+      optional(10, "some_nulls", StringType.get()),
+      optional(11, "no_nulls", StringType.get()),
+      optional(12, "all_nans", DoubleType.get()),
+      optional(13, "some_nans", FloatType.get()),
+      optional(14, "no_nans", DoubleType.get()),
+      optional(15, "struct_not_null", structFieldType),
+      optional(17, "not_in_file", FloatType.get()),
+      optional(18, "no_stats", StringType.get())
+  );
+
+  private static final Types.StructType _structFieldType =
+      Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get()));
+
+  private static final Schema FILE_SCHEMA = new Schema(
+      required(1, "_id", IntegerType.get()),
+      required(2, "_id_long", LongType.get()),
+      required(3, "_id_double", DoubleType.get()),
+      required(4, "_id_float", FloatType.get()),
+      required(5, "_id_binary", StringType.get()),
+      required(6, "_random_binary", StringType.get()),
+      required(7, "_required", StringType.get()),
+      required(8, "_non_bloom", StringType.get()),
+      optional(9, "_all_nulls", LongType.get()),
+      optional(10, "_some_nulls", StringType.get()),
+      optional(11, "_no_nulls", StringType.get()),
+      optional(12, "_all_nans", DoubleType.get()),
+      optional(13, "_some_nans", FloatType.get()),
+      optional(14, "_no_nans", DoubleType.get()),
+      optional(15, "_struct_not_null", _structFieldType),

Review comment:
       I think  we will need a fully covered test which address all the data types from [Type.java](https://github.com/apache/iceberg/blob/90225d6c9413016d611e2ce5eff37db1bc1b4fc5/api/src/main/java/org/apache/iceberg/types/Type.java#L30),  the selected `Integer`, `Double`, `String`, `Float` are not enough.




-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r651396473



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java
##########
@@ -195,11 +196,12 @@ private void startRowGroup() {
     this.nextCheckRecordCount = Math.min(Math.max(recordCount / 2, 100), 10000);
     this.recordCount = 0;
 
-    PageWriteStore pageStore = pageStoreCtorParquet.newInstance(
+    ColumnChunkPageWriteStore pageStore = pageStoreCtorParquet.newInstance(

Review comment:
       Hi, I have split this into 2 PRs:
   Core: Support writing parquet bloom filter #2642
   Core: Support reading parquet bloom filter #2643




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#issuecomment-848411249


   Thanks for the suggesion, I'll split this into several PRs. @rdblue @kbendick 


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r632274583



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java
##########
@@ -0,0 +1,258 @@
+/*
+ * 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.util.Map;
+import java.util.Set;
+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.types.Types.StructType;
+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.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);
+  }
+
+  /**
+   * Test 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 Map<Integer, Boolean> isFallback = null;
+    private Map<Integer, ColumnDescriptor> cols = null;
+    private Map<Integer, ColumnChunkMetaData> columnMetaMap = null;
+
+    private boolean eval(MessageType fileSchema, BlockMetaData rowGroup, BloomFilterReader bloomFilterReader) {
+      this.bloomReader = bloomFilterReader;
+      this.isFallback = Maps.newHashMap();
+      this.cols = Maps.newHashMap();
+      this.columnMetaMap = 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();
+          isFallback.put(id, ParquetUtil.hasNonBloomFilterPages(meta));
+          columnMetaMap.put(id, meta);
+        }
+      }
+
+      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) {
+      throw new IllegalArgumentException(
+          "Bloom filter does not support not expression!");
+    }

Review comment:
       Yes, this exception shouldn't be reached as `RewriteNot` has already been processed. Maybe we can just return `ROWS_MIGHT_MATCH` like other expression as BloomFilter won't help with `not()`




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] jshmchenxi commented on a change in pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
jshmchenxi commented on a change in pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#discussion_r639361400



##########
File path: site/docs/configuration.md
##########
@@ -40,6 +40,9 @@ Iceberg tables support table properties to configure table behavior, like the de
 | write.parquet.dict-size-bytes      | 2097152 (2 MB)     | Parquet dictionary page size                       |
 | write.parquet.compression-codec    | gzip               | Parquet compression codec                          |
 | write.parquet.compression-level    | null               | Parquet compression level                          |
+| write.parquet.bloom-filter-enabled | false | Whether to enable writing bloom filter; It is also possible to enable it for some columns by specifying the column name within the property followed by # |

Review comment:
       > I also think that we need to more carefully consider how to configure Parquet's bloom filters. I would expect what you've added here as table properties to be column specific. Why did you choose global settings. Does this create a bloom filter with the same NDV for all columns?
   
   @rdblue  Yes, `write.parquet.bloom-filter-enabled` and `write.parquet.bloom-filter-expected-ndv` both support column specific settings. We can set `write.parquet.bloom-filter-enabled#user_id=true` and `write.parquet.bloom-filter-expected-ndv#user_id=1000` to just enable bloom filter for column `user_id` with NDV 1000.
   I'll make the doc more complete in new PRs.




-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #2582: Add support for Parquet BloomFilter

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2582:
URL: https://github.com/apache/iceberg/pull/2582#issuecomment-844578561


   @jshmchenxi, I think this should be done in several PRs instead of one. First, we would need to update the Parquet version, then we would want to add read support and finally we would add write support. That will help keep the changes to a size where reviewers can get through them in a reasonable amount of time.
   
   I also think that we need to more carefully consider how to configure Parquet's bloom filters. I would expect what you've added here as table properties to be column specific. Why did you choose global settings. Does this create a bloom filter with the same NDV for all columns?


-- 
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.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org