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

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

yabola commented on code in PR #1023:
URL: https://github.com/apache/parquet-mr/pull/1023#discussion_r1103820351


##########
parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestRowGroupFilterExactly.java:
##########
@@ -0,0 +1,303 @@
+/*
+ * 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.filter2.compat.PredicateEvaluation.BLOCK_CANNOT_MATCH;
+import static 
org.apache.parquet.filter2.compat.PredicateEvaluation.BLOCK_MUST_MATCH;
+import static org.apache.parquet.filter2.predicate.FilterApi.and;
+import static org.apache.parquet.filter2.predicate.FilterApi.binaryColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.doubleColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.eq;
+import static org.apache.parquet.filter2.predicate.FilterApi.gt;
+import static org.apache.parquet.filter2.predicate.FilterApi.gtEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.in;
+import static org.apache.parquet.filter2.predicate.FilterApi.longColumn;
+import static org.apache.parquet.filter2.predicate.FilterApi.lt;
+import static org.apache.parquet.filter2.predicate.FilterApi.ltEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notEq;
+import static org.apache.parquet.filter2.predicate.FilterApi.notIn;
+import static org.apache.parquet.filter2.predicate.FilterApi.or;
+import static org.apache.parquet.hadoop.ParquetFileWriter.Mode.OVERWRITE;
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.junit.After;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import org.apache.parquet.ParquetReadOptions;
+import org.apache.parquet.column.ParquetProperties;
+import org.apache.parquet.filter2.compat.FilterCompat;
+import org.apache.parquet.filter2.compat.PredicateEvaluation;
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+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.util.HadoopInputFile;
+import org.apache.parquet.io.api.Binary;
+
+import com.google.common.collect.Sets;
+
+@RunWith(Parameterized.class)
+public class TestRowGroupFilterExactly {
+  private final Path FILE = createTempFile();
+  private ParquetProperties.WriterVersion WRITER_VERSION;
+  private final Random RANDOM = new Random(42);
+  private final List<PhoneBookWriter.User> DATA = 
Collections.unmodifiableList(generateData(10000));
+  private final long MAX_ID = DATA.size() - 1;
+  private final long MIN_ID = 0;
+  private final TestPredicateEvaluation testEvaluation = new 
TestPredicateEvaluation();
+
+  @Parameterized.Parameters(name = "Run parquet version {index} ")
+  public static Collection<Object[]> params() {
+    return Arrays.asList(
+      new Object[]{ParquetProperties.WriterVersion.PARQUET_1_0},
+      new Object[]{ParquetProperties.WriterVersion.PARQUET_2_0});
+  }
+
+  public TestRowGroupFilterExactly(ParquetProperties.WriterVersion 
WRITER_VERSION) throws IOException {
+    this.WRITER_VERSION = WRITER_VERSION;
+    deleteFile(FILE);
+    writePhoneBookToFile(FILE, this.WRITER_VERSION);
+  }
+
+  @After
+  public void deleteFiles() throws IOException {
+    deleteFile(FILE);
+    testEvaluation.setTestExactPredicate(new 
ArrayList<>(Arrays.asList(BLOCK_MUST_MATCH, BLOCK_CANNOT_MATCH)));
+  }
+
+  @Test
+  public void testFiltering() throws IOException {
+
+    Set<Binary> existValues = new HashSet<>();
+    existValues.add(Binary.fromString("miller"));
+    existValues.add(Binary.fromString("anderson"));
+
+    assertCorrectFiltering(eq(binaryColumn("name"), null));
+    assertCorrectFiltering(eq(binaryColumn("name"), 
Binary.fromString("miller")));
+    assertCorrectFiltering(eq(longColumn("id"), 1234L));
+    assertCorrectFiltering(eq(binaryColumn("name"), 
Binary.fromString("noneExistName")));
+    assertCorrectFiltering(eq(doubleColumn("location.lat"), 99.9));
+
+    assertCorrectFiltering(notEq(binaryColumn("name"), null));
+    assertCorrectFiltering(notEq(binaryColumn("name"), 
Binary.fromString("miller")));
+    assertCorrectFiltering(notEq(binaryColumn("name"), 
Binary.fromString("noneExistName")));
+
+    assertCorrectFiltering(in(binaryColumn("name"), existValues));
+    assertCorrectFiltering(in(binaryColumn("name"), 
Sets.newHashSet(Binary.fromString("miller"),
+      Binary.fromString("noneExistName"), null)));
+
+    assertCorrectFiltering(notIn(binaryColumn("name"),
+      Sets.newHashSet(Binary.fromString("miller"), 
Binary.fromString("anderson"))));
+    assertCorrectFiltering(notIn(binaryColumn("name"),
+      Sets.newHashSet(Binary.fromString("miller"), 
Binary.fromString("noneExistName"), null)));
+
+    assertCorrectFiltering(lt(longColumn("id"), MAX_ID + 1L));
+    assertCorrectFiltering(lt(longColumn("id"), MAX_ID));
+    assertCorrectFiltering(lt(longColumn("id"), 1234L));
+    assertCorrectFiltering(lt(longColumn("id"), MIN_ID));
+    assertCorrectFiltering(lt(longColumn("id"), MIN_ID - 1L));
+    // for dictionary exactly match less than `miller`
+    assertCorrectFiltering(lt(binaryColumn("name"), 
Binary.fromString("ailler")));
+    assertCorrectFiltering(lt(binaryColumn("name"), 
Binary.fromString("miller")));
+
+    assertCorrectFiltering(ltEq(longColumn("id"), MAX_ID + 1L));
+    assertCorrectFiltering(ltEq(longColumn("id"), MAX_ID));
+    assertCorrectFiltering(ltEq(longColumn("id"), 1234L));
+    assertCorrectFiltering(ltEq(longColumn("id"), MIN_ID));
+    assertCorrectFiltering(ltEq(longColumn("id"), MIN_ID - 1L));
+
+    assertCorrectFiltering(gt(longColumn("id"), MAX_ID + 1L));
+    assertCorrectFiltering(gt(longColumn("id"), MAX_ID));
+    assertCorrectFiltering(gt(longColumn("id"), 1234L));
+    assertCorrectFiltering(gt(longColumn("id"), MIN_ID));
+    assertCorrectFiltering(gt(longColumn("id"), MIN_ID - 1L));
+
+    assertCorrectFiltering(gtEq(longColumn("id"), MAX_ID + 1L));
+    assertCorrectFiltering(gtEq(longColumn("id"), MAX_ID));
+    assertCorrectFiltering(gtEq(longColumn("id"), 1234L));
+    assertCorrectFiltering(gtEq(longColumn("id"), MIN_ID));
+    assertCorrectFiltering(gtEq(longColumn("id"), MIN_ID - 1L));
+
+    assertCorrectFiltering(and(eq(binaryColumn("name"), 
Binary.fromString("noneExistName")),
+      lt(longColumn("id"), -99L)));
+    assertCorrectFiltering(and(eq(binaryColumn("name"), 
Binary.fromString("miller")),
+      lt(longColumn("id"), 1234L)));
+    assertCorrectFiltering(and(eq(binaryColumn("name"), 
Binary.fromString("noneExistName")),
+      lt(longColumn("id"), 1234L)));
+
+    assertCorrectFiltering(or(eq(binaryColumn("name"), 
Binary.fromString("noneExistName")),
+      lt(longColumn("id"), -99L)));
+    assertCorrectFiltering(or(eq(binaryColumn("name"), 
Binary.fromString("miller")),
+      lt(longColumn("id"), 1234L)));
+    assertCorrectFiltering(or(eq(binaryColumn("name"), 
Binary.fromString("noneExistName")),
+      lt(longColumn("id"), 1234L)));
+  }
+
+  private void assertCorrectFiltering(FilterPredicate filter) throws 
IOException {
+    ParquetReadOptions readOptions = ParquetReadOptions.builder()
+      .withRecordFilter(FilterCompat.get(filter)).build();
+
+    // simulate the previous behavior, only skip other filters when predicate 
is BLOCK_CANNOT_MATCH
+    
testEvaluation.setTestExactPredicate(Collections.singletonList(BLOCK_CANNOT_MATCH));

Review Comment:
   simulate the previous behavior, only skip other filters when predicate is 
BLOCK_CANNOT_MATCH





> Improve performance when filters in RowGroupFilter can match exactly
> --------------------------------------------------------------------
>
>                 Key: PARQUET-2237
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2237
>             Project: Parquet
>          Issue Type: Improvement
>            Reporter: Mars
>            Priority: Major
>
> If we can accurately judge by the minMax status, we don’t need to load the 
> dictionary from filesystem and compare one by one anymore.
> Similarly , Bloomfilter needs to load from filesystem, it may costs time and 
> memory. If we can exactly determine the existence/nonexistence of the value 
> from minMax or dictionary filters , then we can avoid using Bloomfilter to 
> Improve performance.
> For example,
>  # read data greater than {{x1}} in the block, if minMax in status is all 
> greater than {{{}x1{}}}, then we don't need to read dictionary and compare 
> one by one.
>  # If we already have page dictionaries and have compared one by one, we 
> don't need to read BloomFilter and compare.



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

Reply via email to