huaxingao commented on code in PR #4831:
URL: https://github.com/apache/iceberg/pull/4831#discussion_r878799778
##########
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:
Changed key to `columnPath`. I left `bloomFilterMode` as is because it could
either be `columnBloomFilterEnabled` or `columnBloomFilterNDVs`.
##########
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:
Actually I prefer to test both English and Chinese (for double bytes
characters)
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]