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 2022/05/21 07:39:23 UTC

[GitHub] [iceberg] hililiwei commented on a diff in pull request #4831: Add Parquet Row Group Bloom Filter Support

hililiwei commented on code in PR #4831:
URL: https://github.com/apache/iceberg/pull/4831#discussion_r878653319


##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -292,11 +320,39 @@ public <D> FileAppender<D> build() throws IOException {
             .withRowGroupSize(rowGroupSize)
             .withPageSize(pageSize)
             .withDictionaryPageSize(dictionaryPageSize)
-            .build(),
+            // TODO: add .withMaxBloomFilterBytes(bloomFilterMaxBytes) once ParquetWriter.Builder supports it
+            .withBloomFilterEnabled(bloomFilterEnabled);
+
+        for (Map.Entry<String, String> entry : columnBloomFilterModes.entrySet()) {
+          String col = entry.getKey();
+          String value = entry.getValue();

Review Comment:
   how about `columnPath` \ `enabled`?



##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -292,11 +320,39 @@ public <D> FileAppender<D> build() throws IOException {
             .withRowGroupSize(rowGroupSize)
             .withPageSize(pageSize)
             .withDictionaryPageSize(dictionaryPageSize)
-            .build(),
+            // TODO: add .withMaxBloomFilterBytes(bloomFilterMaxBytes) once ParquetWriter.Builder supports it
+            .withBloomFilterEnabled(bloomFilterEnabled);
+
+        for (Map.Entry<String, String> entry : columnBloomFilterModes.entrySet()) {
+          String col = entry.getKey();
+          String value = entry.getValue();
+          parquetWriteBuilder.withBloomFilterEnabled(col, Boolean.valueOf(value));
+        }
+
+        for (Map.Entry<String, String> entry : columnBloomFilterNDVs.entrySet()) {
+          String col = entry.getKey();
+          String value = entry.getValue();

Review Comment:
   Similar to the above



##########
api/src/main/java/org/apache/iceberg/data/Record.java:
##########
@@ -58,4 +58,22 @@ default Record copy(String field1, Object value1, String field2, Object value2,
     return copy(overwriteValues);
   }
 
+  default Record copy(String field1, Object value1, String field2, Object value2, String field3, Object value3,
+                      String field4, Object value4, String field5, Object value5, String field6, Object value6,
+                      String field7, Object value7, String field8, Object value8, String field9, Object value9,
+                      String field10, Object value10, String field11, Object value11) {
+    Map<String, Object> overwriteValues = Maps.newHashMapWithExpectedSize(9);

Review Comment:
   It's more intuitive to just use 12. Although it's actually 12, even with 9.



##########
spark/v3.2/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderWithBloomFilter.java:
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.spark.source;
+
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.ZoneOffset;
+import java.util.List;
+import org.apache.hadoop.hive.conf.HiveConf;
+import org.apache.iceberg.BaseTable;
+import org.apache.iceberg.CatalogUtil;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.TableMetadata;
+import org.apache.iceberg.TableOperations;
+import org.apache.iceberg.TableProperties;
+import org.apache.iceberg.TestHelpers.Row;
+import org.apache.iceberg.catalog.Namespace;
+import org.apache.iceberg.catalog.TableIdentifier;
+import org.apache.iceberg.data.FileHelpers;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.hive.HiveCatalog;
+import org.apache.iceberg.hive.TestHiveMetastore;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.spark.SparkValueConverter;
+import org.apache.iceberg.types.Types;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.SparkSession;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS;
+
+@RunWith(Parameterized.class)
+public class TestSparkReaderWithBloomFilter {
+
+  protected String tableName = null;
+  protected Table table = null;
+  protected List<Record> records = null;
+  protected DataFile dataFile = null;
+
+  private static TestHiveMetastore metastore = null;
+  protected static SparkSession spark = null;
+  protected static HiveCatalog catalog = null;
+  protected final boolean vectorized;
+  protected final boolean useBloomFilter;
+
+  public TestSparkReaderWithBloomFilter(boolean vectorized, boolean useBloomFilter) {
+    this.vectorized = vectorized;
+    this.useBloomFilter = useBloomFilter;
+  }
+
+  // Schema passed to create tables
+  public static final Schema SCHEMA = new Schema(
+      Types.NestedField.required(1, "id", Types.IntegerType.get()),
+      Types.NestedField.required(2, "id_long", Types.LongType.get()),
+      Types.NestedField.required(3, "id_double", Types.DoubleType.get()),
+      Types.NestedField.required(4, "id_float", Types.FloatType.get()),
+      Types.NestedField.required(5, "id_string", Types.StringType.get()),
+      Types.NestedField.optional(6, "id_boolean", Types.BooleanType.get()),
+      Types.NestedField.optional(7, "id_date", Types.DateType.get()),
+      Types.NestedField.optional(8, "id_timestamp", Types.TimestampType.withZone()),
+      Types.NestedField.optional(9, "id_int_decimal", Types.DecimalType.of(8, 2)),
+      Types.NestedField.optional(10, "id_long_decimal", Types.DecimalType.of(14, 2)),
+      Types.NestedField.optional(11, "id_fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  private static final int INT_MIN_VALUE = 30;
+  private static final int INT_MAX_VALUE = 329;
+  private static final int INT_VALUE_COUNT = INT_MAX_VALUE - INT_MIN_VALUE + 1;
+  private static final long LONG_BASE = 1000L;
+  private static final double DOUBLE_BASE = 10000D;
+  private static final float FLOAT_BASE = 100000F;
+  private static final String BINARY_PREFIX = "BINARY测试_";
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Before
+  public void writeTestDataFile() throws IOException {
+    this.tableName = "test";
+    createTable(tableName, SCHEMA);
+    this.records = Lists.newArrayList();
+
+    // records all use IDs that are in bucket id_bucket=0
+    GenericRecord record = GenericRecord.create(table.schema());
+
+
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      records.add(record.copy(
+          "id", INT_MIN_VALUE + i,
+          "id_long", LONG_BASE + INT_MIN_VALUE + i,
+          "id_double", DOUBLE_BASE + INT_MIN_VALUE + i,
+          "id_float", FLOAT_BASE + INT_MIN_VALUE + i,
+          "id_string", BINARY_PREFIX + (INT_MIN_VALUE + i),
+          "id_boolean", (i % 2 == 0) ? true : false,
+          "id_date",  LocalDate.parse("2021-09-05"),
+          "id_timestamp", Instant.ofEpochMilli(0L).atOffset(ZoneOffset.UTC),
+          "id_int_decimal", new BigDecimal(String.valueOf(77.77)),
+          "id_long_decimal", new BigDecimal(String.valueOf(88.88)),
+          "id_fixed_decimal", new BigDecimal(String.valueOf(99.99))));
+    }
+
+    this.dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), Row.of(0), records);
+
+    table.newAppend()
+        .appendFile(dataFile)
+        .commit();
+  }
+
+  @After
+  public void cleanup() throws IOException {
+    dropTable("test");
+  }
+
+  @Parameterized.Parameters(name = "vectorized = {0}, useBloomFilter = {1}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        {false, false}, {true, false}, {false, true}, {true, true}
+    };
+  }
+
+  @BeforeClass
+  public static void startMetastoreAndSpark() {
+    metastore = new TestHiveMetastore();
+    metastore.start();
+    HiveConf hiveConf = metastore.hiveConf();
+
+    spark = SparkSession.builder()
+        .master("local[2]")
+        .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname))
+        .enableHiveSupport()
+        .getOrCreate();
+
+    catalog = (HiveCatalog)
+        CatalogUtil.loadCatalog(HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf);
+
+    try {
+      catalog.createNamespace(Namespace.of("default"));
+    } catch (AlreadyExistsException ignored) {
+      // the default namespace already exists. ignore the create error
+    }
+  }
+
+  @AfterClass
+  public static void stopMetastoreAndSpark() throws Exception {
+    catalog = null;
+    metastore.stop();
+    metastore = null;
+    spark.stop();
+    spark = null;
+  }
+
+  protected void createTable(String name, Schema schema) {
+    table = catalog.createTable(TableIdentifier.of("default", name), schema);
+    TableOperations ops = ((BaseTable) table).operations();
+    TableMetadata meta = ops.current();
+    ops.commit(meta, meta.upgradeToFormatVersion(2));
+
+    if (useBloomFilter) {
+      table.updateProperties()
+          .set(TableProperties.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED, String.valueOf(useBloomFilter))
+          .commit();
+    }
+
+    table.updateProperties()
+        .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "100")  // to have multiple row groups
+        .commit();
+    if (vectorized) {
+      table.updateProperties()
+          .set(TableProperties.PARQUET_VECTORIZATION_ENABLED, "true")
+          .set(TableProperties.PARQUET_BATCH_SIZE, "4")
+          .commit();
+    }

Review Comment:
   I don't quite understand this. Can we inject these properties directly when the table is created instead of changing it after it is created?
   
   ```
   table = catalog.createTable(TableIdentifier.of("default", name), schema);
   ```
   



##########
core/src/main/java/org/apache/iceberg/TableProperties.java:
##########
@@ -167,6 +167,17 @@ private TableProperties() {
       "write.delete.parquet.row-group-check-max-record-count";
   public static final int PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT_DEFAULT = 10000;
 
+  public static final String DEFAULT_PARQUET_BLOOM_FILTER_ENABLED = "write.parquet.bloom-filter-enabled.default";
+  public static final boolean DEFAULT_PARQUET_BLOOM_FILTER_ENABLED_DEFAULT = false;
+
+  public static final String PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX = "write.parquet.bloom-filter-enabled.column.";
+  public static final String PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX =
+      "write.parquet.bloom-filter-expected-ndv.column.";
+
+  public static final String PARQUET_BLOOM_FILTER_MAX_BYTES = "write.parquet.bloom-filter-max-bytes";
+  public static final int PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT = 1024 * 1024;
+
+

Review Comment:
   Multiple blank lines



##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -292,11 +320,39 @@ public <D> FileAppender<D> build() throws IOException {
             .withRowGroupSize(rowGroupSize)
             .withPageSize(pageSize)
             .withDictionaryPageSize(dictionaryPageSize)
-            .build(),
+            // TODO: add .withMaxBloomFilterBytes(bloomFilterMaxBytes) once ParquetWriter.Builder supports it
+            .withBloomFilterEnabled(bloomFilterEnabled);
+
+        for (Map.Entry<String, String> entry : columnBloomFilterModes.entrySet()) {
+          String col = entry.getKey();
+          String value = entry.getValue();
+          parquetWriteBuilder.withBloomFilterEnabled(col, Boolean.valueOf(value));
+        }
+
+        for (Map.Entry<String, String> entry : columnBloomFilterNDVs.entrySet()) {
+          String col = entry.getKey();
+          String value = entry.getValue();
+          parquetWriteBuilder.withBloomFilterNDV(col, Long.valueOf(value));
+        }
+
+        return new ParquetWriteAdapter<>(
+            parquetWriteBuilder.build(),
             metricsConfig);
       }
     }
 
+    private Map<String, String> getBloomColumnConfigMap(String prefix) {
+      Map<String, String> columnBloomFilterModes = Maps.newHashMap();
+      config.keySet().stream()
+          .filter(key -> key.startsWith(prefix))
+          .forEach(key -> {
+            String columnAlias = key.replaceFirst(prefix, "");
+            String bloomFilterMode = config.get(key);

Review Comment:
   I recommend that keep the same names as above, make it easier to read.



##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -269,19 +282,34 @@ public <D> FileAppender<D> build() throws IOException {
           conf.set(entry.getKey(), entry.getValue());
         }
 
-        ParquetProperties parquetProperties = ParquetProperties.builder()
+        ParquetProperties.Builder propsBuilder = ParquetProperties.builder()
             .withWriterVersion(writerVersion)
             .withPageSize(pageSize)
             .withDictionaryPageSize(dictionaryPageSize)
             .withMinRowCountForPageSizeCheck(rowGroupCheckMinRecordCount)
             .withMaxRowCountForPageSizeCheck(rowGroupCheckMaxRecordCount)
-            .build();
+            .withMaxBloomFilterBytes(bloomFilterMaxBytes)
+            .withBloomFilterEnabled(bloomFilterEnabled);

Review Comment:
   When `bloomFilterEnabled` is true, all fields are turned on unless the fields are specified to be false in `columnBloomFilterModes`, right?



##########
parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java:
##########
@@ -0,0 +1,950 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.util.List;
+import java.util.Random;
+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.relocated.com.google.common.collect.Lists;
+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.iceberg.types.Types.UUIDType;
+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.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
+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, "long", LongType.get()),
+      required(3, "double", DoubleType.get()),
+      required(4, "float", FloatType.get()),
+      required(5, "string", StringType.get()),
+      required(6, "uuid", UUIDType.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()),
+      optional(19, "boolean", Types.BooleanType.get()),
+      optional(20, "time", Types.TimeType.get()),
+      optional(21, "date", Types.DateType.get()),
+      optional(22, "timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "timestamptz", Types.TimestampType.withZone()),
+      optional(24, "binary", Types.BinaryType.get()),
+      optional(25, "int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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, "_long", LongType.get()),
+      required(3, "_double", DoubleType.get()),
+      required(4, "_float", FloatType.get()),
+      required(5, "_string", StringType.get()),
+      required(6, "_uuid", UUIDType.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()),
+      optional(19, "_boolean", Types.BooleanType.get()),
+      optional(20, "_time", Types.TimeType.get()),
+      optional(21, "_date", Types.DateType.get()),
+      optional(22, "_timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "_timestamptz", Types.TimestampType.withZone()),
+      optional(24, "_binary", Types.BinaryType.get()),
+      optional(25, "_int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "_long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "_fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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测试_";

Review Comment:
   to english?



##########
parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java:
##########
@@ -239,6 +245,13 @@ public <D> FileAppender<D> build() throws IOException {
       CompressionCodecName codec = context.codec();
       int rowGroupCheckMinRecordCount = context.rowGroupCheckMinRecordCount();
       int rowGroupCheckMaxRecordCount = context.rowGroupCheckMaxRecordCount();
+      boolean bloomFilterEnabled = PropertyUtil.propertyAsBoolean(config, DEFAULT_PARQUET_BLOOM_FILTER_ENABLED,
+          DEFAULT_PARQUET_BLOOM_FILTER_ENABLED_DEFAULT);
+      int bloomFilterMaxBytes = PropertyUtil.propertyAsInt(config, PARQUET_BLOOM_FILTER_MAX_BYTES,
+          PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT);
+      Map<String, String> columnBloomFilterModes = getBloomColumnConfigMap(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX);
+      Map<String, String> columnBloomFilterNDVs =
+          getBloomColumnConfigMap(PARQUET_BLOOM_FILTER_COLUMN_EXPECTED_NDV_PREFIX);

Review Comment:
   How about getting it from the Context? Just like the arguments above.



##########
parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java:
##########
@@ -0,0 +1,950 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.util.List;
+import java.util.Random;
+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.relocated.com.google.common.collect.Lists;
+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.iceberg.types.Types.UUIDType;
+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.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
+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, "long", LongType.get()),
+      required(3, "double", DoubleType.get()),
+      required(4, "float", FloatType.get()),
+      required(5, "string", StringType.get()),
+      required(6, "uuid", UUIDType.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()),
+      optional(19, "boolean", Types.BooleanType.get()),
+      optional(20, "time", Types.TimeType.get()),
+      optional(21, "date", Types.DateType.get()),
+      optional(22, "timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "timestamptz", Types.TimestampType.withZone()),
+      optional(24, "binary", Types.BinaryType.get()),
+      optional(25, "int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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, "_long", LongType.get()),
+      required(3, "_double", DoubleType.get()),
+      required(4, "_float", FloatType.get()),
+      required(5, "_string", StringType.get()),
+      required(6, "_uuid", UUIDType.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()),
+      optional(19, "_boolean", Types.BooleanType.get()),
+      optional(20, "_time", Types.TimeType.get()),
+      optional(21, "_date", Types.DateType.get()),
+      optional(22, "_timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "_timestamptz", Types.TimestampType.withZone()),
+      optional(24, "_binary", Types.BinaryType.get()),
+      optional(25, "_int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "_long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "_fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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 Instant instant = Instant.parse("2018-10-10T00:00:00.000Z");
+  private static final List<UUID> RANDOM_UUIDS;
+  private static final List<byte[]> RANDOM_BYTES;
+
+  static {
+    RANDOM_UUIDS = Lists.newArrayList();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID());
+    }
+
+    RANDOM_BYTES = Lists.newArrayList();
+    Random rd = new Random();
+    for (int i = 1; i <= INT_VALUE_COUNT; i += 1) {
+      byte[] byteArray = new byte[i];
+      rd.nextBytes(byteArray);
+      RANDOM_BYTES.add(byteArray);
+    }
+  }
+
+  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(DEFAULT_PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_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) {
+        builder.set("_id", INT_MIN_VALUE + i); // min=30, max=79, num-nulls=0
+        builder.set("_long", LONG_BASE + INT_MIN_VALUE + i); // min=130L, max=179L, num-nulls=0
+        builder.set("_double", DOUBLE_BASE + INT_MIN_VALUE + i); // min=1030D, max=1079D, num-nulls=0
+        builder.set("_float", FLOAT_BASE + INT_MIN_VALUE + i); // min=10030F, max=10079F, num-nulls=0
+        builder.set("_string", BINARY_PREFIX + (INT_MIN_VALUE + i)); // min=BINARY测试_30, max=BINARY测试_79, num-nulls=0
+        builder.set("_uuid", RANDOM_UUIDS.get(i)); // required, random uuid, always non-null
+        builder.set("_required", "req"); // required, always non-null
+        builder.set("_non_bloom", RANDOM_UUIDS.get(i)); // bloom filter not enabled
+        builder.set("_all_nulls", null); // never non-null
+        builder.set("_some_nulls", (i % 10 == 0) ? null : "some"); // includes some null values
+        builder.set("_no_nulls", ""); // optional, but always non-null
+        builder.set("_all_nans", Double.NaN); // never non-nan
+        builder.set("_some_nans", (i % 10 == 0) ? Float.NaN : 2F); // includes some nan values
+        builder.set("_no_nans", 3D); // optional, but always non-nan
+        Record structNotNull = new Record(structSchema);
+        structNotNull.put("_int_field", INT_MIN_VALUE + i);
+        builder.set("_struct_not_null", structNotNull); // struct with int
+        builder.set("_no_stats", TOO_LONG_FOR_STATS); // value longer than 4k will produce no stats
+        builder.set("_boolean", (i % 2 == 0) ? true : false);
+        builder.set("_time", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_date", instant.plusSeconds(i * 86400).getEpochSecond());
+        builder.set("_timestamp", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_timestamptz", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_binary", RANDOM_BYTES.get(i));
+        builder.set("_int_decimal", new BigDecimal(String.valueOf(77.77 + i)));
+        builder.set("_long_decimal", new BigDecimal(String.valueOf(88.88 + i)));
+        builder.set("_fixed_decimal", new BigDecimal(String.valueOf(99.99 + i)));
+
+        appender.add(builder.build());
+      }
+    }
+
+    InputFile inFile = Files.localInput(parquetFile);
+
+    ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inFile));
+
+    Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size());
+    rowGroupMetadata = reader.getRowGroups().get(0);
+    parquetSchema = reader.getFileMetaData().getSchema();
+    bloomStore = reader.getBloomFilterDataReader(rowGroupMetadata);
+  }
+
+  @Test
+  public void testNotNull() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("all_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("some_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("no_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("struct_not_null"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testIsNull() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("all_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("some_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("no_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("struct_not_null"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testRequiredColumn() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("required"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: required columns are always non-null", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("required"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertFalse("Should skip: required columns are always non-null", shouldRead);
+  }
+
+  @Test
+  public void testIsNaNs() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("all_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("some_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("no_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testNotNaNs() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("all_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("some_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("no_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testStartsWith() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("non_bloom", "re"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: no bloom", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "re"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "req"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "so"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "reqs"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "somex"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("no_nulls", "xxx"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testMissingColumn() {
+    TestHelpers.assertThrows("Should complain about missing column in expression",
+        ValidationException.class, "Cannot find field 'missing'",
+        () -> new ParquetBloomRowGroupFilter(SCHEMA, lessThan("missing", 5))
+            .shouldRead(parquetSchema, rowGroupMetadata, bloomStore));
+  }
+
+  @Test
+  public void testColumnNotInFile() {
+    Expression[] exprs = new Expression[]{
+        lessThan("not_in_file", 1.0f), lessThanOrEqual("not_in_file", 1.0f),
+        equal("not_in_file", 1.0f), greaterThan("not_in_file", 1.0f),
+        greaterThanOrEqual("not_in_file", 1.0f), notNull("not_in_file"),
+        isNull("not_in_file"), notEqual("not_in_file", 1.0f), in("not_in_file", 1.0f, 2.0f)
+    };
+
+    for (Expression expr : exprs) {
+      boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr)
+          .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+      Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead);
+    }
+  }
+
+  @Test
+  public void testColumnNotBloomFilterEnabled() {
+    Expression[] exprs = new Expression[]{
+        lessThan("non_bloom", "a"), lessThanOrEqual("non_bloom", "a"), equal("non_bloom", "a"),
+        greaterThan("non_bloom", "a"), greaterThanOrEqual("non_bloom", "a"), notNull("non_bloom"),
+        isNull("non_bloom"), notEqual("non_bloom", "a")
+    };
+
+    for (Expression expr : exprs) {
+      boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr)
+          .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+      Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead);
+    }
+  }
+
+  @Test
+  public void testMissingStats() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("no_stats", "a"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertFalse("Should skip: stats are missing but bloom filter is present", shouldRead);
+  }
+
+  @Test
+  public void testNot() {
+    // this test case must use a real predicate, not alwaysTrue(), otherwise binding will simplify it out

Review Comment:
   would you mind explaining it?



##########
parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java:
##########
@@ -0,0 +1,950 @@
+/*
+ * 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.math.BigDecimal;
+import java.time.Instant;
+import java.util.List;
+import java.util.Random;
+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.relocated.com.google.common.collect.Lists;
+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.iceberg.types.Types.UUIDType;
+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.DEFAULT_PARQUET_BLOOM_FILTER_ENABLED;
+import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
+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, "long", LongType.get()),
+      required(3, "double", DoubleType.get()),
+      required(4, "float", FloatType.get()),
+      required(5, "string", StringType.get()),
+      required(6, "uuid", UUIDType.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()),
+      optional(19, "boolean", Types.BooleanType.get()),
+      optional(20, "time", Types.TimeType.get()),
+      optional(21, "date", Types.DateType.get()),
+      optional(22, "timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "timestamptz", Types.TimestampType.withZone()),
+      optional(24, "binary", Types.BinaryType.get()),
+      optional(25, "int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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, "_long", LongType.get()),
+      required(3, "_double", DoubleType.get()),
+      required(4, "_float", FloatType.get()),
+      required(5, "_string", StringType.get()),
+      required(6, "_uuid", UUIDType.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()),
+      optional(19, "_boolean", Types.BooleanType.get()),
+      optional(20, "_time", Types.TimeType.get()),
+      optional(21, "_date", Types.DateType.get()),
+      optional(22, "_timestamp", Types.TimestampType.withoutZone()),
+      optional(23, "_timestamptz", Types.TimestampType.withZone()),
+      optional(24, "_binary", Types.BinaryType.get()),
+      optional(25, "_int_decimal", Types.DecimalType.of(8, 2)),
+      optional(26, "_long_decimal", Types.DecimalType.of(14, 2)),
+      optional(27, "_fixed_decimal", Types.DecimalType.of(31, 2))
+  );
+
+  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 Instant instant = Instant.parse("2018-10-10T00:00:00.000Z");
+  private static final List<UUID> RANDOM_UUIDS;
+  private static final List<byte[]> RANDOM_BYTES;
+
+  static {
+    RANDOM_UUIDS = Lists.newArrayList();
+    for (int i = 0; i < INT_VALUE_COUNT; i += 1) {
+      RANDOM_UUIDS.add(UUID.randomUUID());
+    }
+
+    RANDOM_BYTES = Lists.newArrayList();
+    Random rd = new Random();
+    for (int i = 1; i <= INT_VALUE_COUNT; i += 1) {
+      byte[] byteArray = new byte[i];
+      rd.nextBytes(byteArray);
+      RANDOM_BYTES.add(byteArray);
+    }
+  }
+
+  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(DEFAULT_PARQUET_BLOOM_FILTER_ENABLED, "true")
+        .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "_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) {
+        builder.set("_id", INT_MIN_VALUE + i); // min=30, max=79, num-nulls=0
+        builder.set("_long", LONG_BASE + INT_MIN_VALUE + i); // min=130L, max=179L, num-nulls=0
+        builder.set("_double", DOUBLE_BASE + INT_MIN_VALUE + i); // min=1030D, max=1079D, num-nulls=0
+        builder.set("_float", FLOAT_BASE + INT_MIN_VALUE + i); // min=10030F, max=10079F, num-nulls=0
+        builder.set("_string", BINARY_PREFIX + (INT_MIN_VALUE + i)); // min=BINARY测试_30, max=BINARY测试_79, num-nulls=0
+        builder.set("_uuid", RANDOM_UUIDS.get(i)); // required, random uuid, always non-null
+        builder.set("_required", "req"); // required, always non-null
+        builder.set("_non_bloom", RANDOM_UUIDS.get(i)); // bloom filter not enabled
+        builder.set("_all_nulls", null); // never non-null
+        builder.set("_some_nulls", (i % 10 == 0) ? null : "some"); // includes some null values
+        builder.set("_no_nulls", ""); // optional, but always non-null
+        builder.set("_all_nans", Double.NaN); // never non-nan
+        builder.set("_some_nans", (i % 10 == 0) ? Float.NaN : 2F); // includes some nan values
+        builder.set("_no_nans", 3D); // optional, but always non-nan
+        Record structNotNull = new Record(structSchema);
+        structNotNull.put("_int_field", INT_MIN_VALUE + i);
+        builder.set("_struct_not_null", structNotNull); // struct with int
+        builder.set("_no_stats", TOO_LONG_FOR_STATS); // value longer than 4k will produce no stats
+        builder.set("_boolean", (i % 2 == 0) ? true : false);
+        builder.set("_time", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_date", instant.plusSeconds(i * 86400).getEpochSecond());
+        builder.set("_timestamp", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_timestamptz", instant.plusSeconds(i * 86400).toEpochMilli());
+        builder.set("_binary", RANDOM_BYTES.get(i));
+        builder.set("_int_decimal", new BigDecimal(String.valueOf(77.77 + i)));
+        builder.set("_long_decimal", new BigDecimal(String.valueOf(88.88 + i)));
+        builder.set("_fixed_decimal", new BigDecimal(String.valueOf(99.99 + i)));
+
+        appender.add(builder.build());
+      }
+    }
+
+    InputFile inFile = Files.localInput(parquetFile);
+
+    ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(inFile));
+
+    Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size());
+    rowGroupMetadata = reader.getRowGroups().get(0);
+    parquetSchema = reader.getFileMetaData().getSchema();
+    bloomStore = reader.getBloomFilterDataReader(rowGroupMetadata);
+  }
+
+  @Test
+  public void testNotNull() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("all_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("some_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("no_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("struct_not_null"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testIsNull() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("all_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("some_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("no_nulls"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("struct_not_null"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testRequiredColumn() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNull("required"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: required columns are always non-null", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNull("required"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertFalse("Should skip: required columns are always non-null", shouldRead);
+  }
+
+  @Test
+  public void testIsNaNs() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("all_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("some_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, isNaN("no_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testNotNaNs() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("all_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("some_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, notNaN("no_nans"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testStartsWith() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("non_bloom", "re"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: no bloom", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "re"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "req"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "so"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("required", "reqs"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("some_nulls", "somex"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+
+    shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, startsWith("no_nulls", "xxx"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertTrue("Should read: bloom filter doesn't help", shouldRead);
+  }
+
+  @Test
+  public void testMissingColumn() {
+    TestHelpers.assertThrows("Should complain about missing column in expression",
+        ValidationException.class, "Cannot find field 'missing'",
+        () -> new ParquetBloomRowGroupFilter(SCHEMA, lessThan("missing", 5))
+            .shouldRead(parquetSchema, rowGroupMetadata, bloomStore));
+  }
+
+  @Test
+  public void testColumnNotInFile() {
+    Expression[] exprs = new Expression[]{
+        lessThan("not_in_file", 1.0f), lessThanOrEqual("not_in_file", 1.0f),
+        equal("not_in_file", 1.0f), greaterThan("not_in_file", 1.0f),
+        greaterThanOrEqual("not_in_file", 1.0f), notNull("not_in_file"),
+        isNull("not_in_file"), notEqual("not_in_file", 1.0f), in("not_in_file", 1.0f, 2.0f)
+    };
+
+    for (Expression expr : exprs) {
+      boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr)
+          .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+      Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead);
+    }
+  }
+
+  @Test
+  public void testColumnNotBloomFilterEnabled() {
+    Expression[] exprs = new Expression[]{
+        lessThan("non_bloom", "a"), lessThanOrEqual("non_bloom", "a"), equal("non_bloom", "a"),
+        greaterThan("non_bloom", "a"), greaterThanOrEqual("non_bloom", "a"), notNull("non_bloom"),
+        isNull("non_bloom"), notEqual("non_bloom", "a")
+    };
+
+    for (Expression expr : exprs) {
+      boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, expr)
+          .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+      Assert.assertTrue("Should read: bloom filter cannot be found: " + expr, shouldRead);
+    }
+  }
+
+  @Test
+  public void testMissingStats() {
+    boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("no_stats", "a"))
+        .shouldRead(parquetSchema, rowGroupMetadata, bloomStore);
+    Assert.assertFalse("Should skip: stats are missing but bloom filter is present", shouldRead);
+  }
+
+  @Test
+  public void testNot() {
+    // this test case must use a real predicate, not alwaysTrue(), otherwise binding will simplify it out
+    for (int i = INT_MIN_VALUE - 20; i < INT_MAX_VALUE + 20; i++) {
+      boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA,   not(equal("id", i)))

Review Comment:
   format



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