rdblue commented on a change in pull request #199:
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r429443627



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
##########
@@ -19,51 +19,211 @@
 
 package org.apache.iceberg.orc;
 
+import com.google.common.collect.Maps;
 import java.io.IOException;
-import java.util.Collections;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.Map;
+import java.util.Optional;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.hadoop.HadoopInputFile;
 import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Conversions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+import org.apache.orc.BooleanColumnStatistics;
+import org.apache.orc.ColumnStatistics;
+import org.apache.orc.DateColumnStatistics;
+import org.apache.orc.DecimalColumnStatistics;
+import org.apache.orc.DoubleColumnStatistics;
+import org.apache.orc.IntegerColumnStatistics;
 import org.apache.orc.Reader;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.TimestampColumnStatistics;
+import org.apache.orc.TypeDescription;
 import org.apache.orc.Writer;
 
 public class OrcMetrics {
 
   private OrcMetrics() {
   }
 
+  static final OffsetDateTime EPOCH = 
Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
   public static Metrics fromInputFile(InputFile file) {
     final Configuration config = (file instanceof HadoopInputFile) ?
         ((HadoopInputFile) file).getConf() : new Configuration();
     return fromInputFile(file, config);
   }
 
-  public static Metrics fromInputFile(InputFile file, Configuration config) {
+  static Metrics fromInputFile(InputFile file, Configuration config) {
     try (Reader orcReader = ORC.newFileReader(file, config)) {
-
-      // TODO: implement rest of the methods for ORC metrics
-      // https://github.com/apache/incubator-iceberg/pull/199
-      return new Metrics(orcReader.getNumberOfRows(),
-          null,
-          null,
-          Collections.emptyMap(),
-          null,
-          null);
+      return buildOrcMetrics(orcReader.getNumberOfRows(),
+          orcReader.getSchema(), orcReader.getStatistics());
     } catch (IOException ioe) {
-      throw new RuntimeIOException(ioe, "Failed to read footer of file: %s", 
file);
+      throw new RuntimeIOException(ioe, "Failed to open file: %s", 
file.location());
     }
   }
 
+  private static Metrics buildOrcMetrics(final long numOfRows, final 
TypeDescription orcSchema,
+                                         final ColumnStatistics[] colStats) {
+    final Schema schema = ORCSchemaUtil.convert(orcSchema);
+    Map<Integer, Long> columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+    Map<Integer, Long> valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+    Map<Integer, Long> nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+    Map<Integer, ByteBuffer> lowerBounds = Maps.newHashMap();
+    Map<Integer, ByteBuffer> upperBounds = Maps.newHashMap();
+
+    for (int i = 0; i < colStats.length; i++) {
+      final ColumnStatistics colStat = colStats[i];
+      final TypeDescription orcCol = orcSchema.findSubtype(i);
+      final Optional<Types.NestedField> icebergColOpt = 
ORCSchemaUtil.icebergID(orcCol)
+          .map(schema::findField);
+
+      if (icebergColOpt.isPresent()) {
+        final Types.NestedField icebergCol = icebergColOpt.get();
+        final int fieldId = icebergCol.fieldId();
+
+        if (colStat.hasNull()) {
+          nullCounts.put(fieldId, numOfRows - colStat.getNumberOfValues());

Review comment:
       I don't think this is correct for repeated fields. Here is the javadoc 
for `getNumberOfValues`:
   
   > Get the number of values in this column. It will differ from the number of 
rows because of NULL values and repeated values.
   
   If this is a repeated field, then the number of values could easily be 
larger than the number of rows. I think the number of null values can only be 
recovered when we know that the type is not repeated -- not contained in a map 
or list.
   
   I think this should detect when a column is contained in a map or list and 
suppress its counts. That's okay because we don't have any expressions that 
would use them right now anyway. Expressions can only reference top-level 
fields or fields nested in structs.
   
   An alternative is to use 1 for the null count instead of omitting it from 
the map, but I don't think that's a good idea because it isn't the correct 
value and will cause the value counts to also be wrong.




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