You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2020/10/22 02:31:13 UTC

[GitHub] [iceberg] yyanyy commented on a change in pull request #1641: Add NaN counter to Metrics and implement in Parquet writers

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



##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java
##########
@@ -137,11 +141,20 @@ public void write(int repetitionLevel, T value) {
     public void setColumnStore(ColumnWriteStore columnStore) {
       this.column.setColumnStore(columnStore);
     }
+
+    @Override
+    public Stream<FieldMetrics> metrics() {
+      if (id != null) {
+        return Stream.of(new FieldMetrics(id.intValue(), 0L, 0L, 0L, null, null));

Review comment:
       We may potentially use `ColumnDescriptor desc` for determine id, but felt that explicitly passing `id` could provide better guarantee. Suggestions are welcomed. 

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriter.java
##########
@@ -121,7 +123,7 @@ public void add(T value) {
 
   @Override
   public Metrics metrics() {
-    return ParquetUtil.footerMetrics(writer.getFooter(), metricsConfig);
+    return ParquetUtil.footerMetrics(writer.getFooter(), model.metrics(), inputSchema, metricsConfig);

Review comment:
       We could potentially use the schema from the footer itself in `ParquetUtil` instead of explicitly pass it here; currently have it this way since it's guaranteed that the schema will match. 

##########
File path: data/src/test/java/org/apache/iceberg/parquet/TestParquetMergingMetrics.java
##########
@@ -0,0 +1,176 @@
+/*
+ * 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.parquet;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.data.GenericAppenderFactory;
+import org.apache.iceberg.data.GenericRecord;
+import org.apache.iceberg.data.RandomGenericData;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.io.FileAppender;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.types.Types;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+public class TestParquetMergingMetrics {

Review comment:
       I later noticed that there's a `TestParquetMetrics` as well, will see if I can merge this to that (and eventually to`TestMetrics`).

##########
File path: parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java
##########
@@ -65,28 +68,25 @@
   private ParquetUtil() {
   }
 
-  // Access modifier is package-private, to only allow use from existing tests
-  static Metrics fileMetrics(InputFile file) {
-    return fileMetrics(file, MetricsConfig.getDefault());
-  }
-
   public static Metrics fileMetrics(InputFile file, MetricsConfig metricsConfig) {
     return fileMetrics(file, metricsConfig, null);
   }
 
   public static Metrics fileMetrics(InputFile file, MetricsConfig metricsConfig, NameMapping nameMapping) {
     try (ParquetFileReader reader = ParquetFileReader.open(ParquetIO.file(file))) {
-      return footerMetrics(reader.getFooter(), metricsConfig, nameMapping);
+      return footerMetrics(reader.getFooter(), Stream.empty(), null, metricsConfig, nameMapping);

Review comment:
       This has the similar problem I mentioned in pr description for importing spark table; if the file itself is directly passed in there's not much chance to get the additional metrics tracked by value writers. Currently `fileMetrics` are only used by tests. Do people have suggestions 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.

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



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