[ 
https://issues.apache.org/jira/browse/PARQUET-2251?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17693007#comment-17693007
 ] 

ASF GitHub Bot commented on PARQUET-2251:
-----------------------------------------

wgtmac commented on code in PR #1033:
URL: https://github.com/apache/parquet-mr/pull/1033#discussion_r1116520642


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestStoreBloomFilter.java:
##########
@@ -0,0 +1,132 @@
+/*
+ * 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.parquet.hadoop;
+
+import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE;
+import static 
org.apache.parquet.hadoop.TestBloomFiltering.generateDictionaryData;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.EncodingStats;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.filter2.recordlevel.PhoneBookWriter;
+import org.apache.parquet.hadoop.example.ExampleParquetWriter;
+import org.apache.parquet.hadoop.metadata.BlockMetaData;
+import org.apache.parquet.hadoop.metadata.ColumnChunkMetaData;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+
+@RunWith(Parameterized.class)
+public class TestStoreBloomFilter {
+  private static final Path FILE_V1 = createTempFile("v1");
+  private static final Path FILE_V2 = createTempFile("v2");
+  private static final List<PhoneBookWriter.User> DATA = 
Collections.unmodifiableList(generateDictionaryData(10000));
+  private final Path file;
+  private final String pageVersion;
+
+  public TestStoreBloomFilter(Path file, String pageVersion) {
+    this.file = file;
+    this.pageVersion = pageVersion;
+  }
+
+  @Parameterized.Parameters(name = "Run {index}: parquet {1}")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(
+      new Object[]{FILE_V1, "pageV1"},

Review Comment:
   nit: the name is a little bit confusing. file format version is not same as 
data page version.





> Avoid generating Bloomfilter when all pages of a column are encoded by 
> dictionary
> ---------------------------------------------------------------------------------
>
>                 Key: PARQUET-2251
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2251
>             Project: Parquet
>          Issue Type: Bug
>            Reporter: Mars
>            Priority: Major
>
> In parquet pageV1, even all pages of a column are encoded by dictionary, it 
> will still generate BloomFilter. Actually it is unnecessary to generate 
> BloomFilter and it cost time and occupy storage.
> Parquet pageV2 doesn't generate BloomFilter if all pages of a column are 
> encoded by dictionary,



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to