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/06/14 07:53:01 UTC

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

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