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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may let right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may let right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2244) Dictionary filter may skip row-groups incorrectly when evaluating notIn

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2244:
-

zhongyujiang commented on PR #1028:
URL: https://github.com/apache/parquet-mr/pull/1028#issuecomment-1434445841

   Then I will not open a revert PR, thanks everyone for commenting on this!




> Dictionary filter may skip row-groups incorrectly when evaluating notIn
> ---
>
> Key: PARQUET-2244
> URL: https://issues.apache.org/jira/browse/PARQUET-2244
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
>Affects Versions: 1.12.2
>Reporter: Yujiang Zhong
>Assignee: Yujiang Zhong
>Priority: Major
>
> Dictionary filter may skip row-groups incorrectly when evaluating `notIn` on 
> optional columns with null values. Here is an example:
> Say there is a optional column `c1` with all pages dict encoded, `c1` has and 
> only has two distinct values: ['foo', null],  and the predicate is  `c1 not 
> in ('foo', 'bar')`. 
> Now dictionary filter may skip this row-group that is actually should not be 
> skipped, because there are nulls in the column.
>  
> This is a bug similar to #1510.



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


[GitHub] [parquet-mr] zhongyujiang commented on pull request #1028: PARQUET-2244: Fix notIn for columns with null values

2023-02-17 Thread via GitHub


zhongyujiang commented on PR #1028:
URL: https://github.com/apache/parquet-mr/pull/1028#issuecomment-1434445841

   Then I will not open a revert PR, thanks everyone for commenting on this!


-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



[jira] [Commented] (PARQUET-2240) DateTimeFormatter is used in static context, but not thread safe

2023-02-17 Thread Gang Wu (Jira)


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

Gang Wu commented on PARQUET-2240:
--

Thanks for reporting the issue. Could you please open a PR? [~shani.elha]

> DateTimeFormatter is used in static context, but not thread safe
> 
>
> Key: PARQUET-2240
> URL: https://issues.apache.org/jira/browse/PARQUET-2240
> Project: Parquet
>  Issue Type: Bug
>  Components: parquet-mr
> Environment: Linux, OpenJDK 17 (based on docker image openjdk:17-slim)
>  
>Reporter: Shani Elharrar
>Priority: Trivial
>  Labels: bug
>   Original Estimate: 24h
>  Remaining Estimate: 24h
>
> DateTimeFormatter is used in static context but not thread safe, a formatter 
> instance is created in PrimitiveStringifer.DateStringifier, and 
> DateStringifier is created in static final DATE_STRINGIFIER, 
> TIMESTAMP_MILLIS_STRINGIFIER, TIMESTAMP_MICROS_STRINGIFIER, 
> TIMESTAMP_NANOS_STRINGIFIER, TIMESTAMP_MILLIS_UTC_STRINGIFIER, 
> TIMESTAMP_MICROS_UTC_STRINGIFIER, and TIMESTAMP_NANOS_UTC_STRINGIFIER.
> This causes exceptions like the following to be thrown from parquet-code:
> java.lang.ArrayIndexOutOfBoundsException: Index 633 out of bounds for length 
> 13
> stacktrace:
>     at 
> java.base/sun.util.calendar.BaseCalendar.getCalendarDateFromFixedDate(BaseCalendar.java:457)
>     at 
> java.base/java.util.GregorianCalendar.computeFields(GregorianCalendar.java:2358)
>     at 
> java.base/java.util.GregorianCalendar.computeFields(GregorianCalendar.java:2273)
>     at java.base/java.util.Calendar.setTimeInMillis(Calendar.java:1827)
>     at java.base/java.util.Calendar.setTime(Calendar.java:1793)
>     at java.base/java.text.SimpleDateFormat.format(SimpleDateFormat.java:978)
>     at java.base/java.text.SimpleDateFormat.format(SimpleDateFormat.java:971)
>     at java.base/java.text.DateFormat.format(DateFormat.java:339)
>     at java.base/java.text.Format.format(Format.java:159)
>     at 
> org.apache.parquet.schema.PrimitiveStringifier$DateStringifier.toFormattedString(PrimitiveStringifier.java:265)
>     at 
> org.apache.parquet.schema.PrimitiveStringifier$DateStringifier.stringify(PrimitiveStringifier.java:256)
>     at 
> org.apache.parquet.column.statistics.IntStatistics.stringify(IntStatistics.java:92)
>     at 
> org.apache.parquet.column.statistics.IntStatistics.stringify(IntStatistics.java:25)
>     at 
> org.apache.parquet.column.statistics.Statistics.minAsString(Statistics.java:423)
>     (... unrelated code)
> A simple solution would be to change those from static to non static values.
> I can create a PR if the solution is ok by the maintainers of the library.



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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is  `BLOCK_MIGHT_MATCH` , left & right 
should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+  // if left and right operation all must needs the block, then we must 
take the block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor predicate) {
+Boolean left = or.getLeft().accept(predicate);
+if (left == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+}
+Boolean right = or.getRight().accept(predicate);
+if (right == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+} else if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an && not an || here

Review Comment:
   done





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {
+  // if left and right operation all must needs the block, then we must 
take the block
+  return BLOCK_MUST_MATCH;
+} else {
+  return BLOCK_MIGHT_MATCH;
+}
+  }
+
+  public static Boolean evaluateOr(Operators.Or or, 
FilterPredicate.Visitor predicate) {
+Boolean left = or.getLeft().accept(predicate);
+if (left == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+}
+Boolean right = or.getRight().accept(predicate);
+if (right == BLOCK_MUST_MATCH) {
+  // if left or right operation must need the block, then we must take the 
block
+  return BLOCK_MUST_MATCH;
+} else if (left == BLOCK_CANNOT_MATCH && right == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an && not an || here

Review Comment:
   done



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   In  `StatisticsFilter` left might match (but can't match in 
DictionaryFilter),  right must match -> return might match in StatisticsFilter, 
 return can't match in DictionaryFilter
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   
   
   





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it.
   
   And I add new 
[UT](https://github.com/apache/parquet-mr/pull/1023/files#diff-8915e6fa23018e02c2e79a3f6cc5078a8882f8031022dbdde217fe9bf1d908afR143)
   
   
   



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it
   





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can
+  // drop a chunk of records if we know that either the left or
+  // the right predicate agrees that no matter what we don't
+  // need this chunk.
+  return BLOCK_CANNOT_MATCH;
+}
+Boolean right = and.getRight().accept(predicate);
+if (right == BLOCK_CANNOT_MATCH) {
+  return BLOCK_CANNOT_MATCH;
+} else if (left == BLOCK_MUST_MATCH && right == BLOCK_MUST_MATCH) {

Review Comment:
   if left is `BLOCK_MUST_MATCH` , right is might `BLOCK_MIGHT_MATCH` , left & 
right should be `BLOCK_MIGHT_MATCH`.
   Because in the next filter may right may be `BLOCK_CANNOT_MATCH ` and we 
should drop it
   



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   Thanks for your review, I update comments and add more UT for `And` `Or`





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   Thanks for your review, I update comments and add more UT for `And` `Or`



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



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

2023-02-17 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2237:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   thanks for your review,  added more UT for `And` `Or` to cover





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


[GitHub] [parquet-mr] yabola commented on a diff in pull request #1023: PARQUET-2237 Improve performance when filters in RowGroupFilter can match exactly

2023-02-17 Thread via GitHub


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


##
parquet-hadoop/src/main/java/org/apache/parquet/filter2/compat/PredicateEvaluation.java:
##
@@ -0,0 +1,105 @@
+/*
+ * 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.filter2.compat;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import org.apache.parquet.filter2.predicate.FilterPredicate;
+import org.apache.parquet.filter2.predicate.Operators;
+
+/**
+ * Used in Filters to mark whether we should DROP the block if data matches 
the condition.
+ * If we cannot decide whether the block matches, it will be always safe to 
return BLOCK_MIGHT_MATCH.
+ * We use Boolean Object here to distinguish the value type, please do not 
modify it.
+ */
+public class PredicateEvaluation {
+  /* The block might match, but we cannot decide yet, will check in the other 
filters. */
+  public static final Boolean BLOCK_MIGHT_MATCH = new Boolean(false);
+  /* The block can match for sure. */
+  public static final Boolean BLOCK_MUST_MATCH = new Boolean(false);
+  /* The block can't match for sure */
+  public static final Boolean BLOCK_CANNOT_MATCH = new Boolean(true);
+
+  public static Boolean evaluateAnd(Operators.And and, 
FilterPredicate.Visitor predicate) {
+Boolean left = and.getLeft().accept(predicate);
+if (left == BLOCK_CANNOT_MATCH) {
+  // seems unintuitive to put an || not an && here but we can

Review Comment:
   thanks for your review,  added more UT for `And` `Or` to cover



-- 
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: dev-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org