You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by "ASF GitHub Bot (Jira)" <ji...@apache.org> on 2023/04/22 17:30:00 UTC

[jira] [Commented] (PARQUET-2237) Improve performance when filters in RowGroupFilter can match exactly

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

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

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


##########
parquet-hadoop/src/main/java/org/apache/parquet/filter2/dictionarylevel/DictionaryFilter.java:
##########
@@ -52,14 +53,24 @@ public class DictionaryFilter implements FilterPredicate.Visitor<Boolean> {
   private static final boolean BLOCK_MIGHT_MATCH = false;
   private static final boolean BLOCK_CANNOT_MATCH = true;
 
-  public static boolean canDrop(FilterPredicate pred, List<ColumnChunkMetaData> columns, DictionaryPageReadStore dictionaries) {
+  public static boolean canDrop(FilterPredicate pred, List<ColumnChunkMetaData> columns,
+    DictionaryPageReadStore dictionaries, AtomicBoolean canExactlyDetermine) {

Review Comment:
   This is a public method. We are changing the signature now. Can you add another one to support callers that are still calling the old one?





> 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
>            Assignee: 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)