huaxingao commented on code in PR #6622:
URL: https://github.com/apache/iceberg/pull/6622#discussion_r1115040262


##########
api/src/main/java/org/apache/iceberg/expressions/MaxAggregate.java:
##########
@@ -0,0 +1,80 @@
+/*
+ * 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.expressions;
+
+import java.util.Comparator;
+import org.apache.iceberg.DataFile;
+import org.apache.iceberg.types.Comparators;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Type.PrimitiveType;
+import org.apache.iceberg.types.Types;
+
+public class MaxAggregate<T> extends ValueAggregate<T> {
+  private final int fieldId;
+  private final PrimitiveType type;
+  private final Comparator<T> comparator;
+
+  protected MaxAggregate(BoundTerm<T> term) {
+    super(Operation.MAX, term);
+    Types.NestedField field = term.ref().field();
+    this.fieldId = field.fieldId();
+    this.type = field.type().asPrimitiveType();
+    this.comparator = Comparators.forType(type);
+  }
+
+  @Override
+  protected boolean hasValue(DataFile file) {
+    return file.upperBounds().containsKey(fieldId)
+        || (safeGet(file.valueCounts(), fieldId) != null

Review Comment:
   Here is my simple test:
   ```
   CREATE TABLE t (id LONG, data STRINGT) USING iceberg PARTITIONED BY (id);
   INSERT INTO t VALUES (1, null), (1, null), (1, null), (2, 'cc'), (2, 'dd'), 
(2, 'ee'), (3, 'ff'), (3, 'gg'),(3, null);
   ```
   The data file is
   ```
   
+------------------------+------------------------+----------------+-----------------+
   |upper_bounds            |lower_bounds            |value_counts    
|null_value_counts|
   
+------------------------+------------------------+----------------+-----------------+
   |{1 -> ��������������}         |{1 -> ��������������}         |{1 -> 3, 2 
-> 3}|{1 -> 0, 2 -> 3} |
   |{1 -> ��������������, 2 -> cc}|{1 -> ��������������, 2 -> cc}|{1 -> 1, 2 
-> 1}|{1 -> 0, 2 -> 0} |
   |{1 -> ��������������, 2 -> ee}|{1 -> ��������������, 2 -> dd}|{1 -> 2, 2 
-> 2}|{1 -> 0, 2 -> 0} |
   |{1 -> ��������������, 2 -> gg}|{1 -> ��������������, 2 -> ff}|{1 -> 3, 2 
-> 3}|{1 -> 0, 2 -> 1} |
   
+------------------------+------------------------+----------------+-----------------+
   ```
   For the partition with id = 1, all the values are null. The `upper_bounds` 
doesn't have an entry. So seems it is `||` instead of `&&`?
   
   ```
     protected boolean hasValue(DataFile file) {
       boolean hasBound = file.upperBounds().containsKey(fieldId);
       Long valueCount = safeGet(file.valueCounts(), fieldId);
       Long nullCount = safeGet(file.nullValueCounts(), fieldId);
       boolean boundAllNull = valueCount != null && valueCount > 0 && nullCount 
!= null && nullCount == valueCount;
       return hasBound || boundAllNull;
     }
   ```
   
   I haven't considered the `nan_value_counts` yet. In the case of `nullCount + 
nanCount == valueCount`, we are not pushing down for now. I will have a follow 
up for the NaNs.



-- 
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: issues-unsubscr...@iceberg.apache.org

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


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@iceberg.apache.org
For additional commands, e-mail: issues-h...@iceberg.apache.org

Reply via email to