rdblue commented on a change in pull request #199:
URL: https://github.com/apache/iceberg/pull/199#discussion_r433565834
##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
##########
@@ -20,50 +20,271 @@
package org.apache.iceberg.orc;
import java.io.IOException;
-import java.util.Collections;
+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.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
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.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.relocated.com.google.common.collect.Queues;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+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() {
}
+ private static final OffsetDateTime EPOCH =
Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+ private 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());
}
}
static Metrics fromWriter(Writer writer) {
- // TODO: implement rest of the methods for ORC metrics in
- // https://github.com/apache/incubator-iceberg/pull/199
- return new Metrics(writer.getNumberOfRows(),
- null,
- null,
- Collections.emptyMap(),
- null,
- null);
+ try {
+ return buildOrcMetrics(writer.getNumberOfRows(),
+ writer.getSchema(), writer.getStatistics());
+ } catch (IOException ioe) {
+ throw new RuntimeIOException(ioe, "Failed to get statistics from
writer");
+ }
+ }
+
+ private static Metrics buildOrcMetrics(final long numOfRows, final
TypeDescription orcSchema,
+ final ColumnStatistics[] colStats) {
+ final Schema schema = ORCSchemaUtil.convert(orcSchema);
+ final Set<TypeDescription> columnsInContainers =
findColumnsInContainers(schema, orcSchema);
+ Map<Integer, Long> columnSizes =
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();
+
+ columnSizes.put(fieldId, colStat.getBytesOnDisk());
+
+ if (!columnsInContainers.contains(orcCol)) {
+ if (colStat.hasNull()) {
+ nullCounts.put(fieldId, numOfRows - colStat.getNumberOfValues());
+ } else {
+ nullCounts.put(fieldId, 0L);
+ }
+
+ Optional<ByteBuffer> orcMin = (colStat.getNumberOfValues() > 0) ?
+ fromOrcMin(icebergCol, colStat) : Optional.empty();
+ orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(),
byteBuffer));
+ Optional<ByteBuffer> orcMax = (colStat.getNumberOfValues() > 0) ?
+ fromOrcMax(icebergCol, colStat) : Optional.empty();
+ orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(),
byteBuffer));
+ }
+
+ // Since ORC does not track null values nor repeated ones, the value
count for columns in
+ // containers (maps, list) may be larger than what it actually is,
however these are not
+ // used in experssions right now. For such cases, we use the value
number of values
+ // directly stored in ORC.
Review comment:
I think this should be moved into the `if` block above. If the number of
values required by the spec is not known, it should not be included.
----------------------------------------------------------------
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]