[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-26 Thread GitBox


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



##
File path: 
orc/src/main/java/org/apache/iceberg/orc/avro/GenericDataOrcWriter.java
##
@@ -0,0 +1,526 @@
+/*
+ * 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.avro;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.avro.generic.GenericData;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.common.type.HiveDecimal;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class GenericDataOrcWriter implements 
OrcValueWriter {
+
+  private final Converter[] converters;
+
+  private GenericDataOrcWriter(TypeDescription schema) {
+this.converters = buildConverters(schema);
+  }
+
+  public static OrcValueWriter buildWriter(TypeDescription 
fileSchema) {
+return new GenericDataOrcWriter(fileSchema);
+  }
+
+  @SuppressWarnings("unchecked")
+  @Override
+  public void write(GenericData.Record value, VectorizedRowBatch output) 
throws IOException {
+int row = output.size++;
+for (int c = 0; c < converters.length; ++c) {
+  converters[c].addValue(row, value.get(c), output.cols[c]);
+}
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter {
+
+Class getJavaClass();
+
+/**
+ * Take a value from the Spark data value and add it to the ORC output.
+ * @param rowId the row in the ColumnVector
+ * @param data either an InternalRow or ArrayData
+ * @param output the ColumnVector to put the value into
+ */
+void addValue(int rowId, T data, ColumnVector output);
+  }
+
+  static class BooleanConverter implements Converter {
+@Override
+public Class getJavaClass() {
+  return Boolean.class;
+}
+
+@Override
+public void addValue(int rowId, Boolean data, ColumnVector output) {
+  if (data == null) {
+output.noNulls = false;
+output.isNull[rowId] = true;
+  } else {
+output.isNull[rowId] = false;
+((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+  }
+}
+  }
+
+  static class ByteConverter implements Converter {
+@Override
+public Class getJavaClass() {
+  return Byte.class;
+}
+
+public void addValue(int rowId, Byte data, ColumnVector output) {
+  if (data == null) {
+output.noNulls = false;
+output.isNull[rowId] = true;
+  } else {
+output.isNull[rowId] = false;
+((LongColumnVector) output).vector[rowId] = data;
+  }
+}
+  }
+
+  static class ShortConverter implements Converter {
+@Override
+public Class getJavaClass() {
+  return Short.class;
+}
+
+public void addValue(int rowId, Short data, ColumnVector output) {
+  if (data == null) {
+output.noNulls = false;
+output.isNull[rowId] = true;
+  } else {
+output.isNull[rowId] = false;
+((LongColumnVector) output).vector[rowId] = data;
+  }
+}
+  }
+
+  static class IntConverter implements Converter {
+@Override
+public Class getJavaClass() {
+  return Integer.class;
+}
+
+public void addValue(int rowId, Integer data, ColumnVector output) {
+  if (data == null) {
+output.noNulls 

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: 
parquet/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
##
@@ -57,10 +78,133 @@ public File writeRecords(Schema schema, Map properties, GenericD
 return ParquetWritingTestUtils.writeRecords(temp, schema, properties, 
records);
   }
 
-  @Override
-  public int splitCount(File parquetFile) throws IOException {
+  private int splitCount(File parquetFile) throws IOException {
 try (ParquetFileReader reader = 
ParquetFileReader.open(ParquetIO.file(localInput(parquetFile {
   return reader.getRowGroups().size();
 }
   }
+
+  @Test
+  public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception {

Review comment:
   If it depends on constants that can't easily be translated, then that's 
okay. The rationale is just to keep all the tests that we may eventually want 
to run on other formats in the same test file. This is minor, though.





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
##
@@ -65,10 +65,28 @@ public String name() {
 public TypeDescription type() {
   return type;
 }
+
+@Override
+public boolean equals(Object o) {
+  if (this == o) {
+return true;
+  }
+  if (o == null || getClass() != o.getClass()) {
+return false;
+  }
+
+  OrcField orcField = (OrcField) o;
+  return Objects.equals(name, orcField.name) && Objects.equals(type, 
orcField.type);
+}
+
+@Override
+public int hashCode() {
+  return Objects.hash(name, type);

Review comment:
   Why was it necessary to add `equals` and `hashCode` implementations to 
`ORCField`? I don't see it used in a map or comparison anymore. Maybe this can 
be removed?





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: orc/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
##
@@ -0,0 +1,138 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.util.Map;
+import org.apache.avro.generic.GenericData;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestMetrics;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+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.Conversions.fromByteBuffer;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+/**
+ * Test Metrics for ORC.
+ */
+public class TestOrcMetrics extends TestMetrics {
+
+  static final ImmutableSet BINARY_TYPES = 
ImmutableSet.of(Type.TypeID.BINARY,
+  Type.TypeID.FIXED, Type.TypeID.UUID);
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Override
+  public FileFormat fileFormat() {
+return FileFormat.ORC;
+  }
+
+  @Override
+  public Metrics getMetrics(InputFile file) {
+return OrcMetrics.fromInputFile(file);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, GenericData.Record... records) 
throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, records);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, Map properties,
+   GenericData.Record... records) throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, properties, records);
+  }
+
+  private boolean isBinaryType(Type type) {
+return BINARY_TYPES.contains(type.typeId());
+  }
+
+  @Override
+  protected  void assertBounds(int fieldId, Type type, T lowerBound, T 
upperBound, Metrics metrics) {
+if (isBinaryType(type)) {
+  return;
+}
+
+Map lowerBounds = metrics.lowerBounds();
+Map upperBounds = metrics.upperBounds();
+
+Assert.assertEquals(
+lowerBound,
+lowerBounds.containsKey(fieldId) ? fromByteBuffer(type, 
lowerBounds.get(fieldId)) : null);
+Assert.assertEquals(
+upperBound,
+upperBounds.containsKey(fieldId) ? fromByteBuffer(type, 
upperBounds.get(fieldId)) : null);
+  }
+
+  @Test
+  public void testMetricsForListAndMapElements() throws IOException {
+Types.StructType structType = Types.StructType.of(
+required(1, "leafIntCol", Types.IntegerType.get()),
+optional(2, "leafStringCol", Types.StringType.get())
+);
+Schema schema = new Schema(
+optional(3, "intListCol", Types.ListType.ofRequired(4, 
Types.IntegerType.get())),
+optional(5, "mapCol", Types.MapType.ofRequired(6, 7, 
Types.StringType.get(), structType))
+);
+
+GenericData.Record record = new 
GenericData.Record(AvroSchemaUtil.convert(schema.asStruct()));
+record.put("intListCol", Lists.newArrayList(10, 11, 12));
+GenericData.Record struct = new 
GenericData.Record(AvroSchemaUtil.convert(structType));
+struct.put("leafIntCol", 1);
+struct.put("leafStringCol", "BBB");
+Map map = Maps.newHashMap();
+map.put("4", struct);
+record.put(1, map);
+
+File recordsFile = writeRecords(schema, record);
+
+Metrics metrics = getMetrics(Files.localInput(recordsFile));
+Assert.assertEquals(1L, (long) metrics.recordCount());
+assertCounts(1, 1, 0, metrics);
+assertBounds(1, Types.IntegerType.get(), 1, 1, metrics);
+assertCounts(2, 1, 0, metrics);
+assertBounds(2, Types.StringType.get(),
+CharBuffer.wrap("BBB"), CharBuffer.wr

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: 
parquet/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java
##
@@ -57,10 +78,133 @@ public File writeRecords(Schema schema, Map properties, GenericD
 return ParquetWritingTestUtils.writeRecords(temp, schema, properties, 
records);
   }
 
-  @Override
-  public int splitCount(File parquetFile) throws IOException {
+  private int splitCount(File parquetFile) throws IOException {
 try (ParquetFileReader reader = 
ParquetFileReader.open(ParquetIO.file(localInput(parquetFile {
   return reader.getRowGroups().size();
 }
   }
+
+  @Test
+  public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception {

Review comment:
   I think we should keep these tests in the base file and ignore them for 
ORC using `Assume.assumeTrue(fileFormat() == FileFormat.PARQUET)`.





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: orc/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
##
@@ -0,0 +1,138 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.util.Map;
+import org.apache.avro.generic.GenericData;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestMetrics;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+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.Conversions.fromByteBuffer;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+/**
+ * Test Metrics for ORC.
+ */
+public class TestOrcMetrics extends TestMetrics {
+
+  static final ImmutableSet BINARY_TYPES = 
ImmutableSet.of(Type.TypeID.BINARY,
+  Type.TypeID.FIXED, Type.TypeID.UUID);
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Override
+  public FileFormat fileFormat() {
+return FileFormat.ORC;
+  }
+
+  @Override
+  public Metrics getMetrics(InputFile file) {
+return OrcMetrics.fromInputFile(file);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, GenericData.Record... records) 
throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, records);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, Map properties,
+   GenericData.Record... records) throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, properties, records);
+  }
+
+  private boolean isBinaryType(Type type) {
+return BINARY_TYPES.contains(type.typeId());
+  }
+
+  @Override
+  protected  void assertBounds(int fieldId, Type type, T lowerBound, T 
upperBound, Metrics metrics) {
+if (isBinaryType(type)) {
+  return;

Review comment:
   This should still have assertions. Binary types should not have entries 
in the `metrics.lowerBounds()` or `metrics.upperBounds()` maps, right?





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: orc/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java
##
@@ -0,0 +1,138 @@
+/*
+ * 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.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.CharBuffer;
+import java.util.Map;
+import org.apache.avro.generic.GenericData;
+import org.apache.iceberg.FileFormat;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.TestMetrics;
+import org.apache.iceberg.avro.AvroSchemaUtil;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+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.Conversions.fromByteBuffer;
+import static org.apache.iceberg.types.Types.NestedField.optional;
+import static org.apache.iceberg.types.Types.NestedField.required;
+
+/**
+ * Test Metrics for ORC.
+ */
+public class TestOrcMetrics extends TestMetrics {
+
+  static final ImmutableSet BINARY_TYPES = 
ImmutableSet.of(Type.TypeID.BINARY,
+  Type.TypeID.FIXED, Type.TypeID.UUID);
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Override
+  public FileFormat fileFormat() {
+return FileFormat.ORC;
+  }
+
+  @Override
+  public Metrics getMetrics(InputFile file) {
+return OrcMetrics.fromInputFile(file);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, GenericData.Record... records) 
throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, records);
+  }
+
+  @Override
+  public File writeRecords(Schema schema, Map properties,
+   GenericData.Record... records) throws IOException {
+return OrcWritingTestUtils.writeRecords(temp, schema, properties, records);
+  }
+
+  private boolean isBinaryType(Type type) {
+return BINARY_TYPES.contains(type.typeId());
+  }
+
+  @Override
+  protected  void assertBounds(int fieldId, Type type, T lowerBound, T 
upperBound, Metrics metrics) {
+if (isBinaryType(type)) {
+  return;
+}
+
+Map lowerBounds = metrics.lowerBounds();

Review comment:
   I think this should call `super.assertBounds(...)` so that for 
non-binary types, the checks remain identical.





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: 
orc/src/main/java/org/apache/iceberg/orc/avro/GenericDataOrcWriter.java
##
@@ -0,0 +1,526 @@
+/*
+ * 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.avro;
+
+import com.google.common.collect.Lists;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.avro.generic.GenericData;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.storage.common.type.HiveDecimal;
+import org.apache.orc.storage.ql.exec.vector.BytesColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.ListColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.MapColumnVector;
+import org.apache.orc.storage.ql.exec.vector.StructColumnVector;
+import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+public class GenericDataOrcWriter implements 
OrcValueWriter {

Review comment:
   Could this use Iceberg generics instead? Iceberg generics are already 
supported so we wouldn't need to add a writer for Avro generics.





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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());
+} else {
+  nullCounts.put(fieldId, 0L);
+}
+columSizes.put(fieldId, colStat.getBytesOnDisk());
+valueCounts.put(fieldId, colStat.getNumberOfValues() + 
nullCounts.getOrDefault(fieldId, 0L));
+
+Optional orcMin = (colStat.getNumberOfValues() > 0) ?
+fromOrcMin(icebergCol, colStat) : Optional.empty();
+orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(), 
byteBuffer));
+Optional orcMax = (colStat.getNumberOfValues() > 0) ?
+fromOrcMax(icebergCol, colStat) : Optional.empty();
+orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(), 
byteBuffer));
+  }
+}
+
+return new Metrics(numOfRows,
+columSizes,
+valueCounts,
+nullCounts,
+lowerBounds,
+upperBounds);
+  }
+
   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.getNumberOfRo

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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());
+} else {
+  nullCounts.put(fieldId, 0L);
+}
+columSizes.put(fieldId, colStat.getBytesOnDisk());
+valueCounts.put(fieldId, colStat.getNumberOfValues() + 
nullCounts.getOrDefault(fieldId, 0L));
+
+Optional orcMin = (colStat.getNumberOfValues() > 0) ?
+fromOrcMin(icebergCol, colStat) : Optional.empty();
+orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(), 
byteBuffer));
+Optional orcMax = (colStat.getNumberOfValues() > 0) ?
+fromOrcMax(icebergCol, colStat) : Optional.empty();
+orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(), 
byteBuffer));
+  }
+}
+
+return new Metrics(numOfRows,
+columSizes,
+valueCounts,
+nullCounts,
+lowerBounds,
+upperBounds);
+  }
+
   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.getNumberOfRo

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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());
+} else {
+  nullCounts.put(fieldId, 0L);
+}
+columSizes.put(fieldId, colStat.getBytesOnDisk());
+valueCounts.put(fieldId, colStat.getNumberOfValues() + 
nullCounts.getOrDefault(fieldId, 0L));
+
+Optional orcMin = (colStat.getNumberOfValues() > 0) ?
+fromOrcMin(icebergCol, colStat) : Optional.empty();
+orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(), 
byteBuffer));
+Optional orcMax = (colStat.getNumberOfValues() > 0) ?
+fromOrcMax(icebergCol, colStat) : Optional.empty();
+orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(), 
byteBuffer));
+  }
+}
+
+return new Metrics(numOfRows,
+columSizes,
+valueCounts,
+nullCounts,
+lowerBounds,
+upperBounds);
+  }
+
   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.getNumberOfRo

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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());
+} else {
+  nullCounts.put(fieldId, 0L);
+}
+columSizes.put(fieldId, colStat.getBytesOnDisk());
+valueCounts.put(fieldId, colStat.getNumberOfValues() + 
nullCounts.getOrDefault(fieldId, 0L));
+
+Optional orcMin = (colStat.getNumberOfValues() > 0) ?
+fromOrcMin(icebergCol, colStat) : Optional.empty();
+orcMin.ifPresent(byteBuffer -> lowerBounds.put(icebergCol.fieldId(), 
byteBuffer));
+Optional orcMax = (colStat.getNumberOfValues() > 0) ?
+fromOrcMax(icebergCol, colStat) : Optional.empty();
+orcMax.ifPresent(byteBuffer -> upperBounds.put(icebergCol.fieldId(), 
byteBuffer));
+  }
+}
+
+return new Metrics(numOfRows,
+columSizes,
+valueCounts,
+nullCounts,
+lowerBounds,
+upperBounds);
+  }
+
   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.getNumberOfRo

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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());
+} else {
+  nullCounts.put(fieldId, 0L);
+}
+columSizes.put(fieldId, colStat.getBytesOnDisk());
+valueCounts.put(fieldId, colStat.getNumberOfValues() + 
nullCounts.getOrDefault(fieldId, 0L));

Review comment:
   This should only set the value count if the null count is present. If 
the null count is unknown (repeated fields) then the value count is also 
unknown.





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map nullCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+Map lowerBounds = Maps.newHashMap();
+Map upperBounds = Maps.newHashMap();
+
+for (int i = 0; i < colStats.length; i++) {
+  final ColumnStatistics colStat = colStats[i];
+  final TypeDescription orcCol = orcSchema.findSubtype(i);
+  final Optional 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

[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
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 columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);

Review comment:
   Typo: `columSizes` -> `columnSizes`





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-22 Thread GitBox


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



##
File path: orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java
##
@@ -65,10 +65,28 @@ public String name() {
 public TypeDescription type() {
   return type;
 }
+
+@Override
+public boolean equals(Object o) {
+  if (this == o) {
+return true;
+  }
+  if (o == null || getClass() != o.getClass()) {
+return false;
+  }
+
+  OrcField orcField = (OrcField) o;
+  return Objects.equals(name, orcField.name) && Objects.equals(type, 
orcField.type);
+}
+
+@Override
+public int hashCode() {
+  return Objects.hash(name, type);
+}
   }
 
-  private static final String ICEBERG_ID_ATTRIBUTE = "iceberg.id";
-  private static final String ICEBERG_REQUIRED_ATTRIBUTE = "iceberg.required";
+  static final String ICEBERG_ID_ATTRIBUTE = "iceberg.id";
+  static final String ICEBERG_REQUIRED_ATTRIBUTE = "iceberg.required";

Review comment:
   I don't see where these are used outside of this class. Why make the 
package-private instead of private?





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2020-05-21 Thread GitBox


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



##
File path: gradle/wrapper/gradle-wrapper.properties
##
@@ -1,24 +1,6 @@
-#
-# 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.
-#
-
+#Wed Jan 29 07:37:53 PST 2020
+distributionUrl=https\://services.gradle.org/distributions/gradle-5.4.1-all.zip

Review comment:
   Is this change needed?





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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2019-06-11 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r292555999
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -19,16 +19,32 @@
 
 package org.apache.iceberg.orc;
 
+import static org.apache.iceberg.types.Conversions.toByteBuffer;
 
 Review comment:
   Also, I think that baseline doesn't like static imports for methods, so this 
should just be removed. Instead, use the full `Conversions.toByteBuffer` to 
call it.


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


With regards,
Apache Git Services

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



[GitHub] [incubator-iceberg] rdblue commented on a change in pull request #199: ORC metrics

2019-06-11 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r292555668
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -19,16 +19,32 @@
 
 package org.apache.iceberg.orc;
 
+import static org.apache.iceberg.types.Conversions.toByteBuffer;
 
 Review comment:
   Static imports go after regular imports.


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291774632
 
 

 ##
 File path: orc/src/test/java/org/apache/iceberg/orc/OrcMetricsTest.java
 ##
 @@ -0,0 +1,135 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Conversions;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class OrcMetricsTest {
+  private static final Random RAND = new Random();
+
+  private final TypeDescription orcSchema;
+  private final Schema icebergSchema;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public OrcMetricsTest() {
+orcSchema = 
TypeDescription.fromString("struct");
+icebergSchema = TypeConversion.fromOrc(orcSchema);
+  }
+
+  private File writeOrcTestFile(int rows) throws IOException {
+final Configuration conf = new Configuration();
+
+final File testFile = temp.newFile();
+Assert.assertTrue("Delete should succeed", testFile.delete());
+
+final Path testPath = new Path(testFile.toURI().toString());
+final List optionsLong = ImmutableList.of(10L, 20L, 30L, 100L);
+final List optionsDouble = ImmutableList.of(1.3, 1.7, 0.21, 2.3, 
0.09);
+
+try (Writer writer = OrcFile.createWriter(testPath,
+OrcFile.writerOptions(conf).setSchema(orcSchema))) {
+  VectorizedRowBatch batch = orcSchema.createRowBatch();
+  LongColumnVector w = (LongColumnVector) batch.cols[0];
+  LongColumnVector x = (LongColumnVector) batch.cols[1];
+  LongColumnVector y = (LongColumnVector) batch.cols[2];
+  DoubleColumnVector z = (DoubleColumnVector) batch.cols[3];
+
+  for (int r = 0; r < rows; ++r) {
+int row = batch.size++;
+w.vector[row] = r;
+x.vector[row] = optionsLong.get(RAND.nextInt(optionsLong.size()));
+y.vector[row] = r * 3;
+z.vector[row] = optionsDouble.get(RAND.nextInt(optionsDouble.size()));
+// If the batch is full, write it out and start over.
+if (batch.size == batch.getMaxSize()) {
+  writer.addRowBatch(batch);
+  batch.reset();
+}
+  }
+  if (batch.size != 0) {
+writer.addRowBatch(batch);
+batch.reset();
+  }
+}
+return testFile;
+  }
+
+  @Test
+  public void testOrcMetricsPrimitive() throws IOException {
 
 Review comment:
   @aokolnychyi wrote a thorough test for Parquet. Could this reuse that code 
instead of writing a new one?


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291774523
 
 

 ##
 File path: orc/src/test/java/org/apache/iceberg/orc/OrcMetricsTest.java
 ##
 @@ -0,0 +1,135 @@
+/*
+ * 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 static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Maps;
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.Metrics;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.types.Conversions;
+import org.apache.orc.OrcFile;
+import org.apache.orc.TypeDescription;
+import org.apache.orc.Writer;
+import org.apache.orc.storage.ql.exec.vector.DoubleColumnVector;
+import org.apache.orc.storage.ql.exec.vector.LongColumnVector;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class OrcMetricsTest {
+  private static final Random RAND = new Random();
+
+  private final TypeDescription orcSchema;
+  private final Schema icebergSchema;
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  public OrcMetricsTest() {
+orcSchema = 
TypeDescription.fromString("struct");
+icebergSchema = TypeConversion.fromOrc(orcSchema);
 
 Review comment:
   I think that this test should create an Iceberg schema with non-sequential 
field IDs and convert that to ORC. That would validate that the problems that 
@rdsr pointed out -- using type ID as an index or index as the type ID -- are 
fixed.


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291774284
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/TypeConversion.java
 ##
 @@ -159,21 +154,21 @@ Type convertOrcToType(TypeDescription schema, 
ColumnIdMap columnIds) {
 for (int c=0; c < fieldNames.size(); ++c) {
   String name = fieldNames.get(c);
   TypeDescription type = fieldTypes.get(c);
-  fields.add(Types.NestedField.optional(columnIds.get(type), name,
-  convertOrcToType(type, columnIds)));
+  fields.add(Types.NestedField.optional(type.getId(), name,
 
 Review comment:
   Did ORC add field IDs? Why wasn't `getId` used before?


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291773996
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.iceberg.types.Conversions.toByteBuffer;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+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.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+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.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.TimestampColumnStatistics;
+
+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 Schema schema = TypeConversion.fromOrc(orcReader.getSchema());
+
+  ColumnStatistics[] colStats = orcReader.getStatistics();
+  Map columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map lowerBounds = Maps.newHashMap();
+  Map upperBounds = Maps.newHashMap();
+
+  for(Types.NestedField col : schema.columns()) {
+final int i = col.fieldId();
 
 Review comment:
   The name should be `fieldId`, not `i` because `i` implies it is an index.


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291773906
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.iceberg.types.Conversions.toByteBuffer;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+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.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+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.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.TimestampColumnStatistics;
+
+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 Schema schema = TypeConversion.fromOrc(orcReader.getSchema());
+
+  ColumnStatistics[] colStats = orcReader.getStatistics();
+  Map columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map lowerBounds = Maps.newHashMap();
+  Map upperBounds = Maps.newHashMap();
+
+  for(Types.NestedField col : schema.columns()) {
+final int i = col.fieldId();
+columSizes.put(i, colStats[i].getBytesOnDisk());
+valueCounts.put(i, colStats[i].getNumberOfValues());
 
 Review comment:
   The metrics filters also require null value counts to determine whether any 
or all of the values in a column are null. Is that available from ORC?


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291773630
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -0,0 +1,142 @@
+/*
+ * 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 static org.apache.iceberg.types.Conversions.toByteBuffer;
+
+import com.google.common.collect.Maps;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+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.hadoop.HadoopInputFile;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
+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.OrcFile;
+import org.apache.orc.Reader;
+import org.apache.orc.StringColumnStatistics;
+import org.apache.orc.TimestampColumnStatistics;
+
+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 Schema schema = TypeConversion.fromOrc(orcReader.getSchema());
+
+  ColumnStatistics[] colStats = orcReader.getStatistics();
+  Map columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map lowerBounds = Maps.newHashMap();
+  Map upperBounds = Maps.newHashMap();
+
+  for(Types.NestedField col : schema.columns()) {
+final int i = col.fieldId();
+columSizes.put(i, colStats[i].getBytesOnDisk());
 
 Review comment:
   +1. This needs to use the mapping in the ORC file to get the correct column 
for each field ID.


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


With regards,
Apache Git Services

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



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

2019-06-07 Thread GitBox
rdblue commented on a change in pull request #199: ORC metrics and 
listPartition support
URL: https://github.com/apache/incubator-iceberg/pull/199#discussion_r291773469
 
 

 ##
 File path: orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java
 ##
 @@ -0,0 +1,154 @@
+/*
+ * 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 static org.apache.iceberg.types.Conversions.toByteBuffer;
+
+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.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 colStatsProto = 
orcReader.getOrcProtoFileStatistics();
+  Map columSizes = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map valueCounts = 
Maps.newHashMapWithExpectedSize(colStats.length);
+  Map lowerBounds = Maps.newHashMap();
+  Map upperBounds = Maps.newHashMap();
+
+  for (int i = 0; i < colStats.length; i++) {
+columSizes.put(i, colStats[i].getBytesOnDisk());
 
 Review comment:
   +1, these maps should definitely be keyed by `fieldId`.


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


With regards,
Apache Git Services

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