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 2019/05/29 00:13:37 UTC

[GitHub] [incubator-iceberg] rdsr commented on a change in pull request #199: ORC metrics and listPartition support

rdsr commented on a change in pull request #199: ORC metrics and listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r288349511
 
 

 ##########
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##########
 @@ -0,0 +1,162 @@
+/*
+ * 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.orc;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.exceptions.RuntimeIOException;
+import org.apache.iceberg.expressions.Literal;
+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.ColumnStatistics;
+import org.apache.orc.OrcFile;
+import org.apache.orc.OrcProto;
+import org.apache.orc.Reader;
+import org.apache.orc.TypeDescription;
+
+public class OrcMetrics {
+
+  private OrcMetrics() {}
+
+  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) {
+    try {
+      final Reader orcReader = OrcFile.createReader(new Path(file.location()),
+          OrcFile.readerOptions(config));
+      final TypeDescription orcSchema = orcReader.getSchema();
+      final Schema schema = TypeConversion.fromOrc(orcSchema);
+
+      ColumnStatistics[] colStats = orcReader.getStatistics();
+      List<OrcProto.ColumnStatistics> colStatsProto = orcReader.getOrcProtoFileStatistics();
+      Map<Integer, Long> columSizes = Maps.newHashMapWithExpectedSize(colStats.length);
+      Map<Integer, Long> valueCounts = Maps.newHashMapWithExpectedSize(colStats.length);
+      Map<Integer, Literal<?>> lowerBounds = Maps.newHashMap();
+      Map<Integer, Literal<?>> upperBounds = Maps.newHashMap();
+
+      for (int i = 0; i < colStats.length; i++) {
+        columSizes.put(i, colStats[i].getBytesOnDisk());
+        valueCounts.put(i, colStats[i].getNumberOfValues());
+
+        final OrcProto.ColumnStatistics protoStats = colStatsProto.get(i);
+        final Types.NestedField col = schema.findField(i);
+        if (col != null) {
+          Optional<Literal<?>> orcMin = fromOrcMin(col, protoStats);
+          if (orcMin.isPresent()) {
+            lowerBounds.put(i, orcMin.get());
+          }
+          Optional<Literal<?>> orcMax = fromOrcMax(col, protoStats);
+          if (orcMax.isPresent()) {
+            upperBounds.put(i, orcMax.get());
+          }
+        }
+      }
+
+      return new Metrics(orcReader.getNumberOfRows(),
+          columSizes,
+          valueCounts,
+          Maps.newHashMap(),
+          toBufferMap(schema, lowerBounds),
+          toBufferMap(schema, upperBounds));
+    } catch (IOException ioe) {
+      throw new RuntimeIOException(ioe, "Failed to read footer of file: %s", file);
+    }
+  }
+
+  private static Optional<Literal<?>> fromOrcMin(Types.NestedField column,
+                                                    OrcProto.ColumnStatistics columnStats) {
+    Literal<?> min = null;
+    if (columnStats.hasIntStatistics()) {
+      if (column.type().typeId() == Type.TypeID.INTEGER) {
+        min = Literal.of((int) columnStats.getIntStatistics().getMinimum());
 
 Review comment:
   Why convert to bytebuffers in a separate function? It seems that if we do it here itself, we save traversing the columns again and possibly also simplify the code a little.
   e.g instead of 
   `Literal.of(columnStats.getDoubleStatistics().getMaximum());` we could do 
   ` Conversions.toByteBuffer(column.type(), columnStats.getDoubleStatistics().getMinimum());`
     

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


With regards,
Apache Git Services

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