yyanyy commented on a change in pull request #1829:
URL: https://github.com/apache/iceberg/pull/1829#discussion_r537791995



##########
File path: core/src/main/java/org/apache/iceberg/MetricsUtil.java
##########
@@ -0,0 +1,48 @@
+/*
+ * 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;
+
+import java.util.Map;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+
+public class MetricsUtil {
+
+  private MetricsUtil() {
+  }
+
+  public static Map<Integer, Long> getNanValueCounts(
+      Stream<FieldMetrics> fieldMetrics, MetricsConfig metricsConfig, Schema 
inputSchema) {
+    if (fieldMetrics == null || inputSchema == null) {
+      return Maps.newHashMap();
+    }
+
+    return fieldMetrics
+        .filter(metrics -> getMetricsMode(inputSchema, metricsConfig, 
metrics.id()) != MetricsModes.None.get())
+        .collect(Collectors.toMap(FieldMetrics::id, 
FieldMetrics::nanValueCount));
+  }
+
+  public static MetricsModes.MetricsMode getMetricsMode(Schema inputSchema, 
MetricsConfig metricsConfig, int fieldId) {
+    String columnName = inputSchema.findColumnName(fieldId);

Review comment:
       Yes, sorry I overwrote my own earlier change that addressed this when I 
renamed this file... 

##########
File path: core/src/main/java/org/apache/iceberg/NaNFieldMetrics.java
##########
@@ -17,51 +17,50 @@
  * under the License.
  */
 
-package org.apache.iceberg.parquet;
+package org.apache.iceberg;
 
 import java.nio.ByteBuffer;
-import org.apache.iceberg.FieldMetrics;
 
 /**
- * Iceberg internally tracked field level metrics, used by Parquet writer only.
+ * Iceberg internally tracked field level metrics, used by Parquet and ORC 
writers only.
  * <p>
- * Parquet keeps track of most metrics in its footer, and only NaN counter is 
actually tracked by writers.
- * This wrapper ensures that metrics not being updated by Parquet writers will 
not be incorrectly used, by throwing
+ * Parquet/ORC keeps track of most metrics in file statistics, and only NaN 
counter is actually tracked by writers.
+ * This wrapper ensures that metrics not being updated by those writers will 
not be incorrectly used, by throwing
  * exceptions when they are accessed.
  */
-public class ParquetFieldMetrics extends FieldMetrics {
+public class NaNFieldMetrics extends FieldMetrics {

Review comment:
       Sounds good; I'll keep the javadoc as is since they currently reflect 
reality, and will update comments to mention tracking float fields when the 
class starts to track upper/lower bound 




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

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to