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/13 21:02:43 UTC

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

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