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

[GitHub] [iceberg] openinx opened a new pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

openinx opened a new pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455329081



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {

Review comment:
       Missing parameter type.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458233371



##########
File path: data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java
##########
@@ -180,7 +180,7 @@ public void createOrcInputFile(List<Record> records) throws IOException {
     OutputFile outFile = Files.localOutput(ORC_FILE);
     try (FileAppender<Record> appender = ORC.write(outFile)
         .schema(FILE_SCHEMA)
-        .createWriterFunc(GenericOrcWriter::buildWriter)
+        .createWriterFunc(typeDesc -> GenericOrcWriter.buildWriter(FILE_SCHEMA, typeDesc))

Review comment:
       Since the schema is passed to the write builder, what about adding a `createWriterFunc` method that accepts `BiFunction<Schema, TypeDescription>`? Then this wouldn't need to change.
   
   We do this in Avro: https://github.com/apache/iceberg/blob/master/core/src/main/java/org/apache/iceberg/avro/Avro.java#L199-L203
   
   That would cut down on the number of files that need to change in this PR.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-664098589


   > @rdsr I've refactored the SparkOrcWrier by using OrcSchemaWithTypeVisitor in [here](https://github.com/apache/iceberg/pull/1238/files#diff-5aa4840155ec70fdf7f725e122cde7b7R44), we can see that the constructor of SparkOrcWriter will also need the two arguments: iceberg schema and TypeDescription. So actually, although we could add a `Function` in here, but we actually won't use it in the newly refactored spark orc writer, that's why I say we don't need to introduce the redundant `Function` method.
   
   @openinx That makes sense to me. Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454089900



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter<Byte> {
+    private static final Converter<Byte> INSTANCE = new ByteConverter();
+
+    @Override
+    public Class<Byte> getJavaClass() {
+      return Byte.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter<Short> {
+    private static final Converter<Short> INSTANCE = new ShortConverter();
+
+    @Override
+    public Class<Short> getJavaClass() {
+      return Short.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter<Integer> {
+    private static final Converter<Integer> INSTANCE = new IntConverter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Integer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class TimeConverter implements Converter<LocalTime> {
+    private static final Converter<LocalTime> INSTANCE = new TimeConverter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter<Long> {
+    private static final Converter<Long> INSTANCE = new LongConverter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Long data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter<Float> {
+    private static final Converter<Float> INSTANCE = new FloatConverter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Float data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter<Double> {
+    private static final Converter<Double> INSTANCE = new DoubleConverter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Double data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class StringConverter implements Converter<String> {
+    private static final Converter<String> INSTANCE = new StringConverter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void addValue(int rowId, String data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        byte[] value = data.getBytes(StandardCharsets.UTF_8);
+        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+      }
+    }
+  }
+
+  private static class BytesConverter implements Converter<ByteBuffer> {
+    private static final Converter<ByteBuffer> INSTANCE = new BytesConverter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+      }
+    }
+  }
+
+  private static class UUIDConverter implements Converter<UUID> {
+    private static final Converter<UUID> INSTANCE = new UUIDConverter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void addValue(int rowId, UUID data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ByteBuffer buffer = ByteBuffer.allocate(16);
+        buffer.putLong(data.getMostSignificantBits());
+        buffer.putLong(data.getLeastSignificantBits());
+        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+      }
+    }
+  }
+
+  private static class FixedConverter implements Converter<byte[]> {
+    private static final Converter<byte[]> INSTANCE = new FixedConverter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void addValue(int rowId, byte[] data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+      }
+    }
+  }
+
+  private static class DateConverter implements Converter<LocalDate> {
+    private static final Converter<LocalDate> INSTANCE = new DateConverter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDate data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+      }
+    }
+  }
+
+  private static class TimestampTzConverter implements Converter<OffsetDateTime> {
+    private static final Converter<OffsetDateTime> INSTANCE = new TimestampTzConverter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter<LocalDateTime> {
+    private static final Converter<LocalDateTime> INSTANCE = new TimestampConverter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.setIsUTC(true);
+        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class Decimal18Converter implements Converter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Converter(TypeDescription schema) {
+      this.scale = schema.getScale();
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId]
+            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+      }
+    }
+  }
+
+  private static class Decimal38Converter implements Converter<BigDecimal> {
+    private static final Converter<BigDecimal> INSTANCE = new Decimal38Converter();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+      }
+    }
+  }
+
+  public static class RecordConverter implements Converter<Record> {
+    private final List<Converter> converters;
+
+    RecordConverter(List<Converter> converters) {
+      this.converters = converters;
+    }
+
+    public List<Converter> converters() {
+      return converters;
+    }
+
+    @Override
+    public Class<Record> getJavaClass() {
+      return Record.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Record data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        StructColumnVector cv = (StructColumnVector) output;
+        for (int c = 0; c < converters.size(); ++c) {
+          converters.get(c).addValue(rowId, data.get(c, converters.get(c).getJavaClass()), cv.fields[c]);
+        }
+      }
+    }
+  }
+
+  public static class ListConverter implements Converter<List> {
+    private final Converter children;
+
+    ListConverter(Converter children) {
+      this.children = children;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, List data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        List<Object> value = (List<Object>) data;
+        ListColumnVector cv = (ListColumnVector) output;
+        // record the length and start of the list elements
+        cv.lengths[rowId] = value.size();
+        cv.offsets[rowId] = cv.childCount;
+        cv.childCount += cv.lengths[rowId];
+        // make sure the child is big enough
+        cv.child.ensureSize(cv.childCount, true);
+        // Add each element
+        for (int e = 0; e < cv.lengths[rowId]; ++e) {
+          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+        }
+      }
+    }
+  }
+
+  public static class MapConverter implements Converter<Map> {
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+
+    MapConverter(Converter keyConverter, Converter valueConverter) {
+      this.keyConverter = keyConverter;
+      this.valueConverter = valueConverter;
+    }
+
+    @Override
+    public Class<Map> getJavaClass() {
+      return Map.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Map data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        Map<Object, Object> map = (Map<Object, Object>) data;
+        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
+        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
+        for (Map.Entry<?, ?> entry : map.entrySet()) {
+          keys.add(entry.getKey());
+          values.add(entry.getValue());
+        }
+        MapColumnVector cv = (MapColumnVector) output;

Review comment:
       It's true, but seems hacky to abstract the similar partten into a separate method, I'd prefer to keep the current implementation.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue merged pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue merged pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454180115



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java
##########
@@ -146,8 +146,8 @@ private static Writer newOrcWriter(OutputFile file,
   }
 
   @SuppressWarnings("unchecked")
-  private static <D> OrcValueWriter<D> newOrcValueWriter(
-      TypeDescription schema, Function<TypeDescription, OrcValueWriter<?>> createWriterFunc) {
-    return (OrcValueWriter<D>) createWriterFunc.apply(schema);
+  private static <D> OrcRowWriter<D> newOrcValueWriter(

Review comment:
       Thanks for your reminding, Good point.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459241319



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,414 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);

Review comment:
       nit: scala -> scale :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454031494



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter<Byte> {
+    private static final Converter<Byte> INSTANCE = new ByteConverter();
+
+    @Override
+    public Class<Byte> getJavaClass() {
+      return Byte.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter<Short> {
+    private static final Converter<Short> INSTANCE = new ShortConverter();
+
+    @Override
+    public Class<Short> getJavaClass() {
+      return Short.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter<Integer> {
+    private static final Converter<Integer> INSTANCE = new IntConverter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Integer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class TimeConverter implements Converter<LocalTime> {
+    private static final Converter<LocalTime> INSTANCE = new TimeConverter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter<Long> {
+    private static final Converter<Long> INSTANCE = new LongConverter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Long data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter<Float> {
+    private static final Converter<Float> INSTANCE = new FloatConverter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Float data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter<Double> {
+    private static final Converter<Double> INSTANCE = new DoubleConverter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Double data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class StringConverter implements Converter<String> {
+    private static final Converter<String> INSTANCE = new StringConverter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void addValue(int rowId, String data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        byte[] value = data.getBytes(StandardCharsets.UTF_8);
+        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+      }
+    }
+  }
+
+  private static class BytesConverter implements Converter<ByteBuffer> {
+    private static final Converter<ByteBuffer> INSTANCE = new BytesConverter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+      }
+    }
+  }
+
+  private static class UUIDConverter implements Converter<UUID> {
+    private static final Converter<UUID> INSTANCE = new UUIDConverter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void addValue(int rowId, UUID data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ByteBuffer buffer = ByteBuffer.allocate(16);
+        buffer.putLong(data.getMostSignificantBits());
+        buffer.putLong(data.getLeastSignificantBits());
+        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+      }
+    }
+  }
+
+  private static class FixedConverter implements Converter<byte[]> {
+    private static final Converter<byte[]> INSTANCE = new FixedConverter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void addValue(int rowId, byte[] data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+      }
+    }
+  }
+
+  private static class DateConverter implements Converter<LocalDate> {
+    private static final Converter<LocalDate> INSTANCE = new DateConverter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDate data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+      }
+    }
+  }
+
+  private static class TimestampTzConverter implements Converter<OffsetDateTime> {
+    private static final Converter<OffsetDateTime> INSTANCE = new TimestampTzConverter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter<LocalDateTime> {
+    private static final Converter<LocalDateTime> INSTANCE = new TimestampConverter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.setIsUTC(true);
+        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class Decimal18Converter implements Converter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Converter(TypeDescription schema) {
+      this.scale = schema.getScale();
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId]
+            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+      }
+    }
+  }
+
+  private static class Decimal38Converter implements Converter<BigDecimal> {
+    private static final Converter<BigDecimal> INSTANCE = new Decimal38Converter();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+      }
+    }
+  }
+
+  public static class RecordConverter implements Converter<Record> {
+    private final List<Converter> converters;
+
+    RecordConverter(List<Converter> converters) {
+      this.converters = converters;
+    }
+
+    public List<Converter> converters() {
+      return converters;
+    }
+
+    @Override
+    public Class<Record> getJavaClass() {
+      return Record.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Record data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        StructColumnVector cv = (StructColumnVector) output;
+        for (int c = 0; c < converters.size(); ++c) {
+          converters.get(c).addValue(rowId, data.get(c, converters.get(c).getJavaClass()), cv.fields[c]);
+        }
+      }
+    }
+  }
+
+  public static class ListConverter implements Converter<List> {
+    private final Converter children;
+
+    ListConverter(Converter children) {
+      this.children = children;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, List data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        List<Object> value = (List<Object>) data;
+        ListColumnVector cv = (ListColumnVector) output;
+        // record the length and start of the list elements
+        cv.lengths[rowId] = value.size();
+        cv.offsets[rowId] = cv.childCount;
+        cv.childCount += cv.lengths[rowId];
+        // make sure the child is big enough
+        cv.child.ensureSize(cv.childCount, true);
+        // Add each element
+        for (int e = 0; e < cv.lengths[rowId]; ++e) {
+          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+        }
+      }
+    }
+  }
+
+  public static class MapConverter implements Converter<Map> {
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+
+    MapConverter(Converter keyConverter, Converter valueConverter) {
+      this.keyConverter = keyConverter;
+      this.valueConverter = valueConverter;
+    }
+
+    @Override
+    public Class<Map> getJavaClass() {
+      return Map.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Map data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        Map<Object, Object> map = (Map<Object, Object>) data;
+        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
+        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
+        for (Map.Entry<?, ?> entry : map.entrySet()) {
+          keys.add(entry.getKey());
+          values.add(entry.getValue());
+        }
+        MapColumnVector cv = (MapColumnVector) output;

Review comment:
       It seems filling a columnVector using a converter is a pattern which can be extracted across map keys, map values and list ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-664049255


   @openinx
   I felt adding a `BiFunction` was better than replacing it. And regarding 
   > That makes more confuse and the OrcFileAppender need to choose the non-null function to create the OrcRowWriter
   `Avro` handles that well. See `org.apache.iceberg.avro.Avro.ReadBuilder#build`
   But I'm ok with the patch in its current form.  Since @rdblue initiated this comment. I'll wait for his final approval before merging.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454061777



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       I don't think `OrcValueReader` and `OrcValueWriter` are public interfaces meant to be used by users of Iceberg. They are meant for internal Iceberg usage




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455281054



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -434,175 +144,12 @@ public void addValue(int rowId, BigDecimal data, ColumnVector output) {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
+    public void nonNullWrite(int rowId, Record data, ColumnVector output) {
+      StructColumnVector cv = (StructColumnVector) output;
+      for (int c = 0; c < writers.size(); ++c) {
+        OrcValueWriter child = writers.get(c);
+        child.write(rowId, data.get(c, child.getJavaClass()), cv.fields[c]);
       }
     }
   }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();

Review comment:
       > I'd prefer to pass the LocalDateTime object to the comparator and do the LocalDateTime to Long conversion when comparing
   
   Iceberg's internal representation does not use higher-level types like LocalDateTime for a few good reasons:
   1. It is simpler to work with ordinal values
   2. The interpretation of an ordinal value is delegated to the object model: Iceberg is agnostic to calendars, time zones, and other concerns that are built into the processing engines
   3. The guarantee is simpler: whatever data values are passed into Iceberg will be passed back out, unmodified




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454018001



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       Any benefit of using GenericOrcWriters.Converter<?>?

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       We should call this  `OrcValueWriter` and rename existing `OrcValueWriter` to `OrcRowWriter` to preserve symmetry between  `OrcValueReader` and `OrcRowWriter`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-663985352


   Ping @rdblue @rdsr ,  Mind to take a look again and help to merge this PR ?  Thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454025116



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);
+        case TIMESTAMP:
+          return GenericOrcWriters.timestamp();
+        case TIMESTAMP_INSTANT:
+          return GenericOrcWriters.timestampTz();
+      }
+      throw new IllegalArgumentException("Unhandled type " + schema);
+    }
   }
 
   @SuppressWarnings("unchecked")
   @Override
-  public void write(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, converters[c].getJavaClass()), output.cols[c]);
-    }
-  }
-
-  /**
-   * The interface for the conversion from Spark's SpecializedGetters to
-   * ORC's ColumnVectors.
-   */
-  interface Converter<T> {
-
-    Class<T> 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<Boolean> {
-    @Override
-    public Class<Boolean> 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<Byte> {
-    @Override
-    public Class<Byte> getJavaClass() {
-      return Byte.class;
-    }
-
-    @Override
-    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<Short> {
-    @Override
-    public Class<Short> getJavaClass() {
-      return Short.class;
-    }
-
-    @Override
-    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<Integer> {
-    @Override
-    public Class<Integer> getJavaClass() {
-      return Integer.class;
-    }
-
-    @Override
-    public void addValue(int rowId, Integer 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 TimeConverter implements Converter<LocalTime> {
-    @Override
-    public Class<LocalTime> getJavaClass() {
-      return LocalTime.class;
-    }
-
-    @Override
-    public void addValue(int rowId, LocalTime data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
-      }
-    }
-  }
-
-  static class LongConverter implements Converter<Long> {
-    @Override
-    public Class<Long> getJavaClass() {
-      return Long.class;
-    }
-
-    @Override
-    public void addValue(int rowId, Long 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 FloatConverter implements Converter<Float> {
-    @Override
-    public Class<Float> getJavaClass() {
-      return Float.class;
-    }
-
-    @Override
-    public void addValue(int rowId, Float data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((DoubleColumnVector) output).vector[rowId] = data;
-      }
-    }
-  }
-
-  static class DoubleConverter implements Converter<Double> {
-    @Override
-    public Class<Double> getJavaClass() {
-      return Double.class;
-    }
-
-    @Override
-    public void addValue(int rowId, Double data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((DoubleColumnVector) output).vector[rowId] = data;
-      }
-    }
-  }
-
-  static class StringConverter implements Converter<String> {
-    @Override
-    public Class<String> getJavaClass() {
-      return String.class;
-    }
-
-    @Override
-    public void addValue(int rowId, String data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        byte[] value = data.getBytes(StandardCharsets.UTF_8);
-        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
-      }
-    }
-  }
-
-  static class BytesConverter implements Converter<ByteBuffer> {
-    @Override
-    public Class<ByteBuffer> getJavaClass() {
-      return ByteBuffer.class;
-    }
-
-    @Override
-    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
-      }
-    }
-  }
-
-  static class UUIDConverter implements Converter<UUID> {
-    @Override
-    public Class<UUID> getJavaClass() {
-      return UUID.class;
-    }
-
-    @Override
-    public void addValue(int rowId, UUID data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ByteBuffer buffer = ByteBuffer.allocate(16);
-        buffer.putLong(data.getMostSignificantBits());
-        buffer.putLong(data.getLeastSignificantBits());
-        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
-      }
-    }
-  }
+  public void write(Record value, VectorizedRowBatch output) {
+    Preconditions.checkArgument(converter instanceof GenericOrcWriters.RecordConverter,
+        "Converter must be a RecordConverter.");
 
-  static class FixedConverter implements Converter<byte[]> {
-    @Override
-    public Class<byte[]> getJavaClass() {
-      return byte[].class;
-    }
-
-    @Override
-    public void addValue(int rowId, byte[] data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
-      }
-    }
-  }
-
-  static class DateConverter implements Converter<LocalDate> {
-    @Override
-    public Class<LocalDate> getJavaClass() {
-      return LocalDate.class;
-    }
-
-    @Override
-    public void addValue(int rowId, LocalDate data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
-      }
-    }
-  }
-
-  static class TimestampTzConverter implements Converter<OffsetDateTime> {
-    @Override
-    public Class<OffsetDateTime> getJavaClass() {
-      return OffsetDateTime.class;
-    }
-
-    @Override
-    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        TimestampColumnVector cv = (TimestampColumnVector) output;
-        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
-        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
-      }
-    }
-  }
-
-  static class TimestampConverter implements Converter<LocalDateTime> {
-
-    @Override
-    public Class<LocalDateTime> getJavaClass() {
-      return LocalDateTime.class;
-    }
-
-    @Override
-    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        TimestampColumnVector cv = (TimestampColumnVector) output;
-        cv.setIsUTC(true);
-        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
-        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
-      }
-    }
-  }
-
-  static class Decimal18Converter implements Converter<BigDecimal> {
-    private final int scale;
-
-    Decimal18Converter(TypeDescription schema) {
-      this.scale = schema.getScale();
-    }
-
-    @Override
-    public Class<BigDecimal> getJavaClass() {
-      return BigDecimal.class;
-    }
-
-    @Override
-    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
-      // TODO: validate precision and scale from schema
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((DecimalColumnVector) output).vector[rowId]
-            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
-      }
-    }
-  }
-
-  static class Decimal38Converter implements Converter<BigDecimal> {
-    Decimal38Converter(TypeDescription schema) {
-    }
-
-    @Override
-    public Class<BigDecimal> getJavaClass() {
-      return BigDecimal.class;
-    }
-
-    @Override
-    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
-      // TODO: validate precision and scale from schema
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
-      }
-    }
-  }
-
-  static class StructConverter implements Converter<Record> {
-    private final Converter[] children;
-
-    StructConverter(TypeDescription schema) {
-      this.children = new Converter[schema.getChildren().size()];
-      for (int c = 0; c < children.length; ++c) {
-        children[c] = buildConverter(schema.getChildren().get(c));
-      }
-    }
-
-    @Override
-    public Class<Record> getJavaClass() {
-      return Record.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
-      }
-    }
-  }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();
-          case BINARY:
-            return new BytesConverter();
-          default:
-            throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
-        }
-      case STRING:
-      case CHAR:
-      case VARCHAR:
-        return new StringConverter();
-      case DECIMAL:
-        return schema.getPrecision() <= 18 ? new Decimal18Converter(schema) : new Decimal38Converter(schema);
-      case TIMESTAMP:
-        return new TimestampConverter();
-      case TIMESTAMP_INSTANT:
-        return new TimestampTzConverter();
-      case STRUCT:
-        return new StructConverter(schema);
-      case LIST:
-        return new ListConverter(schema);
-      case MAP:
-        return new MapConverter(schema);
-    }
-    throw new IllegalArgumentException("Unhandled type " + schema);
-  }
-
-  private static Converter[] buildConverters(TypeDescription schema) {
-    if (schema.getCategory() != TypeDescription.Category.STRUCT) {
-      throw new IllegalArgumentException("Top level must be a struct " + schema);
-    }
-
-    List<TypeDescription> children = schema.getChildren();
-    Converter[] result = new Converter[children.size()];
-    for (int c = 0; c < children.size(); ++c) {
-      result[c] = buildConverter(children.get(c));
+    int row = output.size++;

Review comment:
       instead of ++ I think the recommendation is break it into two statements
   row = output.size
   output.size += 1
   

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);
+        case TIMESTAMP:
+          return GenericOrcWriters.timestamp();
+        case TIMESTAMP_INSTANT:
+          return GenericOrcWriters.timestampTz();
+      }
+      throw new IllegalArgumentException("Unhandled type " + schema);
+    }
   }
 
   @SuppressWarnings("unchecked")

Review comment:
       I guess this suppress unchecked can be removed once we paramterize the converter with a wildcard e.g converter -> converter<?>

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter<Byte> {
+    private static final Converter<Byte> INSTANCE = new ByteConverter();
+
+    @Override
+    public Class<Byte> getJavaClass() {
+      return Byte.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter<Short> {
+    private static final Converter<Short> INSTANCE = new ShortConverter();
+
+    @Override
+    public Class<Short> getJavaClass() {
+      return Short.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter<Integer> {
+    private static final Converter<Integer> INSTANCE = new IntConverter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Integer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class TimeConverter implements Converter<LocalTime> {
+    private static final Converter<LocalTime> INSTANCE = new TimeConverter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter<Long> {
+    private static final Converter<Long> INSTANCE = new LongConverter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Long data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter<Float> {
+    private static final Converter<Float> INSTANCE = new FloatConverter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Float data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter<Double> {
+    private static final Converter<Double> INSTANCE = new DoubleConverter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Double data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class StringConverter implements Converter<String> {
+    private static final Converter<String> INSTANCE = new StringConverter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void addValue(int rowId, String data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        byte[] value = data.getBytes(StandardCharsets.UTF_8);
+        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+      }
+    }
+  }
+
+  private static class BytesConverter implements Converter<ByteBuffer> {
+    private static final Converter<ByteBuffer> INSTANCE = new BytesConverter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+      }
+    }
+  }
+
+  private static class UUIDConverter implements Converter<UUID> {
+    private static final Converter<UUID> INSTANCE = new UUIDConverter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void addValue(int rowId, UUID data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ByteBuffer buffer = ByteBuffer.allocate(16);
+        buffer.putLong(data.getMostSignificantBits());
+        buffer.putLong(data.getLeastSignificantBits());
+        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+      }
+    }
+  }
+
+  private static class FixedConverter implements Converter<byte[]> {
+    private static final Converter<byte[]> INSTANCE = new FixedConverter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void addValue(int rowId, byte[] data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+      }
+    }
+  }
+
+  private static class DateConverter implements Converter<LocalDate> {
+    private static final Converter<LocalDate> INSTANCE = new DateConverter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDate data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+      }
+    }
+  }
+
+  private static class TimestampTzConverter implements Converter<OffsetDateTime> {
+    private static final Converter<OffsetDateTime> INSTANCE = new TimestampTzConverter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter<LocalDateTime> {
+    private static final Converter<LocalDateTime> INSTANCE = new TimestampConverter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.setIsUTC(true);
+        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class Decimal18Converter implements Converter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Converter(TypeDescription schema) {
+      this.scale = schema.getScale();
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId]
+            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+      }
+    }
+  }
+
+  private static class Decimal38Converter implements Converter<BigDecimal> {
+    private static final Converter<BigDecimal> INSTANCE = new Decimal38Converter();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+      }
+    }
+  }
+
+  public static class RecordConverter implements Converter<Record> {
+    private final List<Converter> converters;
+
+    RecordConverter(List<Converter> converters) {
+      this.converters = converters;
+    }
+
+    public List<Converter> converters() {
+      return converters;
+    }
+
+    @Override
+    public Class<Record> getJavaClass() {
+      return Record.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Record data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        StructColumnVector cv = (StructColumnVector) output;
+        for (int c = 0; c < converters.size(); ++c) {
+          converters.get(c).addValue(rowId, data.get(c, converters.get(c).getJavaClass()), cv.fields[c]);
+        }
+      }
+    }
+  }
+
+  public static class ListConverter implements Converter<List> {
+    private final Converter children;
+
+    ListConverter(Converter children) {
+      this.children = children;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, List data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        List<Object> value = (List<Object>) data;
+        ListColumnVector cv = (ListColumnVector) output;
+        // record the length and start of the list elements
+        cv.lengths[rowId] = value.size();
+        cv.offsets[rowId] = cv.childCount;
+        cv.childCount += cv.lengths[rowId];
+        // make sure the child is big enough
+        cv.child.ensureSize(cv.childCount, true);
+        // Add each element
+        for (int e = 0; e < cv.lengths[rowId]; ++e) {
+          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+        }
+      }
+    }
+  }
+
+  public static class MapConverter implements Converter<Map> {
+    private final Converter keyConverter;

Review comment:
       keeping these parameterize on wild card may help u remove the unchecked annotation

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter<Byte> {
+    private static final Converter<Byte> INSTANCE = new ByteConverter();
+
+    @Override
+    public Class<Byte> getJavaClass() {
+      return Byte.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter<Short> {
+    private static final Converter<Short> INSTANCE = new ShortConverter();
+
+    @Override
+    public Class<Short> getJavaClass() {
+      return Short.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter<Integer> {
+    private static final Converter<Integer> INSTANCE = new IntConverter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Integer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class TimeConverter implements Converter<LocalTime> {
+    private static final Converter<LocalTime> INSTANCE = new TimeConverter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter<Long> {
+    private static final Converter<Long> INSTANCE = new LongConverter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Long data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter<Float> {
+    private static final Converter<Float> INSTANCE = new FloatConverter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Float data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter<Double> {
+    private static final Converter<Double> INSTANCE = new DoubleConverter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Double data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class StringConverter implements Converter<String> {
+    private static final Converter<String> INSTANCE = new StringConverter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void addValue(int rowId, String data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        byte[] value = data.getBytes(StandardCharsets.UTF_8);
+        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+      }
+    }
+  }
+
+  private static class BytesConverter implements Converter<ByteBuffer> {
+    private static final Converter<ByteBuffer> INSTANCE = new BytesConverter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+      }
+    }
+  }
+
+  private static class UUIDConverter implements Converter<UUID> {
+    private static final Converter<UUID> INSTANCE = new UUIDConverter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void addValue(int rowId, UUID data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ByteBuffer buffer = ByteBuffer.allocate(16);
+        buffer.putLong(data.getMostSignificantBits());
+        buffer.putLong(data.getLeastSignificantBits());
+        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+      }
+    }
+  }
+
+  private static class FixedConverter implements Converter<byte[]> {
+    private static final Converter<byte[]> INSTANCE = new FixedConverter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void addValue(int rowId, byte[] data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+      }
+    }
+  }
+
+  private static class DateConverter implements Converter<LocalDate> {
+    private static final Converter<LocalDate> INSTANCE = new DateConverter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDate data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+      }
+    }
+  }
+
+  private static class TimestampTzConverter implements Converter<OffsetDateTime> {
+    private static final Converter<OffsetDateTime> INSTANCE = new TimestampTzConverter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter<LocalDateTime> {
+    private static final Converter<LocalDateTime> INSTANCE = new TimestampConverter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.setIsUTC(true);
+        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class Decimal18Converter implements Converter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Converter(TypeDescription schema) {
+      this.scale = schema.getScale();
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId]
+            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+      }
+    }
+  }
+
+  private static class Decimal38Converter implements Converter<BigDecimal> {
+    private static final Converter<BigDecimal> INSTANCE = new Decimal38Converter();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+      }
+    }
+  }
+
+  public static class RecordConverter implements Converter<Record> {
+    private final List<Converter> converters;
+
+    RecordConverter(List<Converter> converters) {
+      this.converters = converters;
+    }
+
+    public List<Converter> converters() {
+      return converters;
+    }
+
+    @Override
+    public Class<Record> getJavaClass() {
+      return Record.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Record data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        StructColumnVector cv = (StructColumnVector) output;
+        for (int c = 0; c < converters.size(); ++c) {
+          converters.get(c).addValue(rowId, data.get(c, converters.get(c).getJavaClass()), cv.fields[c]);
+        }
+      }
+    }
+  }
+
+  public static class ListConverter implements Converter<List> {
+    private final Converter children;
+
+    ListConverter(Converter children) {
+      this.children = children;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, List data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        List<Object> value = (List<Object>) data;
+        ListColumnVector cv = (ListColumnVector) output;
+        // record the length and start of the list elements
+        cv.lengths[rowId] = value.size();
+        cv.offsets[rowId] = cv.childCount;
+        cv.childCount += cv.lengths[rowId];
+        // make sure the child is big enough
+        cv.child.ensureSize(cv.childCount, true);
+        // Add each element
+        for (int e = 0; e < cv.lengths[rowId]; ++e) {
+          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+        }
+      }
+    }
+  }
+
+  public static class MapConverter implements Converter<Map> {
+    private final Converter keyConverter;
+    private final Converter valueConverter;
+
+    MapConverter(Converter keyConverter, Converter valueConverter) {
+      this.keyConverter = keyConverter;
+      this.valueConverter = valueConverter;
+    }
+
+    @Override
+    public Class<Map> getJavaClass() {
+      return Map.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Map data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        Map<Object, Object> map = (Map<Object, Object>) data;
+        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
+        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
+        for (Map.Entry<?, ?> entry : map.entrySet()) {
+          keys.add(entry.getKey());
+          values.add(entry.getValue());
+        }
+        MapColumnVector cv = (MapColumnVector) output;

Review comment:
       It seems filling a columnVector using a convert is a pattern which can be extracted across map keys, map values and list ?

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);

Review comment:
       +1

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {

Review comment:
       should the switch use iPrimitive ?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458230021



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {

Review comment:
       We still need to add the parameterized types.
   
   The FAQ entry you pointed to explains why there is no class literal, like `List<String>.class`. All variants of `List` use `List.class` because there is only one concrete type at runtime. But we still want to use type parameters to be explicit about what is passed around.
   
   This class handles lists of some type, `T`. The class should be parameterized by T so that we can use type-safe operations to pass around `T`. The wrapped value writer should be `OrcValueWriter<T> elementWriter`. By doing this, the implementation of `nonNullWrite` will get a `List<T>` and will be able to pass those values to the `elementWriter` without casting.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455328354



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);
+        case TIMESTAMP:
+          return GenericOrcWriters.timestamp();
+        case TIMESTAMP_INSTANT:
+          return GenericOrcWriters.timestampTz();
+      }
+      throw new IllegalArgumentException("Unhandled type " + schema);
+    }
   }
 
   @SuppressWarnings("unchecked")

Review comment:
       +1
   
   We should avoid using types without parameters.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-662608271


   Thanks @openinx ! I will have another look, today.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr edited a comment on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr edited a comment on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-664049255


   @openinx
   I felt adding a `BiFunction` was better than replacing it. And regarding 
   > That makes more confuse and the OrcFileAppender need to choose the non-null function to create the OrcRowWriter
   
   `Avro` handles that well. See `org.apache.iceberg.avro.Avro.ReadBuilder#build`
   But I'm ok with the patch in its current form.  Since @rdblue initiated this comment. I'll wait for his final approval before merging.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r457804242



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);
+        case TIMESTAMP:
+          return GenericOrcWriters.timestamp();
+        case TIMESTAMP_INSTANT:
+          return GenericOrcWriters.timestampTz();
+      }
+      throw new IllegalArgumentException("Unhandled type " + schema);
+    }
   }
 
   @SuppressWarnings("unchecked")

Review comment:
       Here I did not change to use `OrcValueWriter<?>` because  if we do then we have the following to write child:
   ```java
   for (int c = 0; c < writers.size(); ++c) {
         OrcValueWriter<?> child = writers.get(c);
         child.write(row, value.get(c, child.getJavaClass()), output.cols[c]);
   }
   ```
   
   The value is a `StructLike` and the  `get` in `StructLike` is `<T> T get(int pos, Class<T> javaClass)`, while child.getClass is a class like `OrcValueWriter<?>.class`, it will throw the compile error: 
   ```
   Incompatible types. Required capture of ? but 'get' was inferred to T: no instance(s) of of type variables(s) exist so that capture of ? conforms to capture of ?
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-657957120


   @rdsr @shardulm94   Mind to take another look ? I've updated the pull request, Thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454563885



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -434,175 +144,12 @@ public void addValue(int rowId, BigDecimal data, ColumnVector output) {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
+    public void nonNullWrite(int rowId, Record data, ColumnVector output) {
+      StructColumnVector cv = (StructColumnVector) output;
+      for (int c = 0; c < writers.size(); ++c) {
+        OrcValueWriter child = writers.get(c);
+        child.write(rowId, data.get(c, child.getJavaClass()), cv.fields[c]);
       }
     }
   }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();

Review comment:
       `TypeID#javaClass()` does not define the Java types exposed by the `iceberg-data` module. They are mainly used internally in `iceberg-core` when serializing and deserializing to/from manifest files and in Expressions as literals. Iceberg Generics use `byte[]` for fixed types. So this looks correct and is in parity with Parquet and Avro Generics. You can also look at Timestamp types. Iceberg Generics uses `LocalDateTime` and `OffsetDateTime` for Timestamp Without Zone and Timestamp With Zone respectively. However both types map to longs in `TypeID#javaClass()`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-665924015


   Looks good. Thanks, @openinx!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-664166246


   Thanks for the confirmation, so if no other concern, please help to merge this PR so that we could move the following flink ORC reader writer work forward.  Thanks in advance. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-658678893


   Ping @rdsr @shardulm94 @rdblue , any other concern ?  Thanks.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-665681586


   @Simon0806   we won't need to abstract the `BaseOrcWriter` again in this patch because we have [discussed](https://github.com/apache/iceberg/pull/1158#discussion_r451323369) to share the common data type writers in the GenericOrcWriter, FlinkOrcWriter, SparkOrcWriter.  That's helpful to decouple the codes for different compute engine.  Thanks.  
   
   BTW, ping @rdblue to merge this patch, Thanks. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454017745



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       We should call this  `OrcValueWriter` and rename existing `OrcValueWriter` to `OrcRowWriter` to preserve symmetry between  `OrcValueReader` and `OrcRowReader`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454107579



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java
##########
@@ -146,8 +146,8 @@ private static Writer newOrcWriter(OutputFile file,
   }
 
   @SuppressWarnings("unchecked")
-  private static <D> OrcValueWriter<D> newOrcValueWriter(
-      TypeDescription schema, Function<TypeDescription, OrcValueWriter<?>> createWriterFunc) {
-    return (OrcValueWriter<D>) createWriterFunc.apply(schema);
+  private static <D> OrcRowWriter<D> newOrcValueWriter(

Review comment:
       Nit: Change method name to reflect class name change




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458230021



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {

Review comment:
       We still need to add the parameterized types.
   
   The FAQ entry you pointed to explains why there is no class literal, like `List<String>.class`. All variants of `List` use `List.class` because there is only one concrete type at runtime. But we still want to use type parameters to be explicit about what is passed around.
   
   This class handles lists of some type, `T`. The class should be parameterized by `T` so that we can use type-safe operations to pass around `T` instances. The wrapped value writer should be `OrcValueWriter<T> elementWriter`. By doing this, the implementation of `nonNullWrite` will get a `List<T>` and will be able to pass those values to the `elementWriter` without casting.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454019685



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,

Review comment:
       we should add the correct type parameter in the return value . E.g for record we should do  `OrcValueWriter<Record>` and similarly for map and list




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459243768



##########
File path: orc/src/main/java/org/apache/iceberg/orc/ORC.java
##########
@@ -143,7 +144,7 @@ private ReadBuilder(InputFile file) {
     /**
      * Restricts the read to the given range: [start, start + length).
      *
-     * @param newStart the start position for this read
+     * @param newStart  the start position for this read

Review comment:
       nit: non functional change




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-661552598


   @rdblue @rdsr  I think this patch is ready to merge now,  please merge this if no other concern. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459241319



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,414 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);

Review comment:
       nit: scala to scale :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459249268



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcRowWriter.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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 java.io.IOException;
+import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+
+/**
+ * Write data value of a schema.

Review comment:
       nit: maybe reword as ..
   writes or appends  a `row` to Orc's `VectorizedRowBatch`

##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -42,7 +42,7 @@
  * This class acts as an adaptor from an OrcFileAppender to a
  * FileAppender&lt;InternalRow&gt;.
  */
-public class SparkOrcWriter implements OrcValueWriter<InternalRow> {
+public class SparkOrcWriter implements OrcRowWriter<InternalRow> {
 
   private final Converter[] converters;

Review comment:
       Does it make sense to file a ticket to refactor SparkOrcWriter to utilize similar approach of using `OrchSchemaVisitor`. This could help with reuse of common generic writers defined in `GenericOrcWriters`, similar to how we did in `SparkOrcReaders`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455463756



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java
##########
@@ -51,12 +51,12 @@
   private final OutputFile file;
   private final Writer writer;
   private final VectorizedRowBatch batch;
-  private final OrcValueWriter<D> valueWriter;
+  private final OrcRowWriter<D> valueWriter;

Review comment:
       Pls see the discussion here: https://github.com/apache/iceberg/pull/1197#discussion_r454017745

##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -65,9 +65,10 @@ public void write(InternalRow value, VectorizedRowBatch output) {
   interface Converter {
     /**
      * Take a value from the Spark data value and add it to the ORC output.
-     * @param rowId the row in the ColumnVector
+     *
+     * @param rowId  the row in the ColumnVector
      * @param column either the column number or element number
-     * @param data either an InternalRow or ArrayData
+     * @param data   either an InternalRow or ArrayData

Review comment:
       OK, I guess I did a code format before.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {

Review comment:
       I did not provide the explicit parameter type here, because  [getJavaClass()](https://github.com/apache/iceberg/pull/1197/files#diff-b1b07b15f036000a3f2bed76fdd9f961R357)  will need to return a  `List` class with generic type, while Java don't support this now.   Pls see [here](http://www.angelikalanger.com/GenericsFAQ/FAQSections/ParameterizedTypes.html#FAQ106)

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {
+    private final OrcValueWriter element;
+
+    ListWriter(OrcValueWriter element) {
+      this.element = element;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void nonNullWrite(int rowId, List data, ColumnVector output) {
+      List<Object> value = (List<Object>) data;
+      ListColumnVector cv = (ListColumnVector) output;
+      // record the length and start of the list elements
+      cv.lengths[rowId] = value.size();
+      cv.offsets[rowId] = cv.childCount;
+      cv.childCount += cv.lengths[rowId];
+      // make sure the child is big enough
+      cv.child.ensureSize(cv.childCount, true);
+      // Add each element
+      for (int e = 0; e < cv.lengths[rowId]; ++e) {
+        element.write((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+      }
+    }
+  }
+
+  private static class MapWriter implements OrcValueWriter<Map> {

Review comment:
       ditto.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459248613



##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -42,7 +42,7 @@
  * This class acts as an adaptor from an OrcFileAppender to a
  * FileAppender&lt;InternalRow&gt;.
  */
-public class SparkOrcWriter implements OrcValueWriter<InternalRow> {
+public class SparkOrcWriter implements OrcRowWriter<InternalRow> {
 
   private final Converter[] converters;

Review comment:
       Does it make sense to file a ticket to refactor SparkOrcWriter to utilize similar approach of using `OrchSchemaVisitor`. This could help with reuse of common generic writers defined in `GenericOrcWriters`, similar to how we did in `SparkOrcReaders`?
   
   We can tackle this ticket in future PRs 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458233957



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java
##########
@@ -51,12 +51,12 @@
   private final OutputFile file;
   private final Writer writer;
   private final VectorizedRowBatch batch;
-  private final OrcValueWriter<D> valueWriter;
+  private final OrcRowWriter<D> valueWriter;

Review comment:
       Okay, sounds good to me.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] shardulm94 commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
shardulm94 commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r453861043



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();

Review comment:
       Iceberg would not produce a TypeDescription with BYTE or SHORT types, so we should just throw an unsupported exception for these cases.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }

Review comment:
       Now that we had the Iceberg primitive type available, we can use the Iceberg type to decide whether this is UUID, FIXED or BINARY. That is similar to what we do in `GenericOrcReader`.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);

Review comment:
       Nit: Can we just pass in the (precision, scale) to the writer instead of the whole schema?

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.

Review comment:
       Remove references to Spark and SpecializedGetters.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> getJavaClass();
+
+    /**
+     * Take a value from the Spark data value and add it to the ORC output.

Review comment:
       Remove references to Spark.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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

Review comment:
       Remove references to Spark's InternalRow and ArrayData classes

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);

Review comment:
       Now that we had the Iceberg primitive type available, we can use the Iceberg type to decide whether this is long or time. That is similar to what we do in `GenericOrcReader`.

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);

Review comment:
       Since we are rewriting this, should we consider putting the `if (precision < 18) then THIS else THAT` logic in `GenericOrcWriters.decimals()` and not exposing Decimal18 and Decimal38 separately? This similar to what we did recently for readers in this PR https://github.com/apache/iceberg/blob/7d0cf1cafec9ff42d8e4cb5019185af00ff80d58/spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcValueReaders.java#L57

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;

Review comment:
       This code is repeated for every converter. Consider doing something similar to the read side, where we put this logic in this Converter interface itself so the the implementations only need to think about non-null cases.
   https://github.com/apache/iceberg/blob/809697a6bf56698b1bc52c5f039f2a190a58fed5/orc/src/main/java/org/apache/iceberg/orc/OrcValueReader.java#L25-L36

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,612 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.data.Record;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  /**
+   * The interface for the conversion from Spark's SpecializedGetters to
+   * ORC's ColumnVectors.
+   */
+  interface Converter<T> {
+
+    Class<T> 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);
+  }
+
+  public static Converter<Boolean> booleans() {
+    return BooleanConverter.INSTANCE;
+  }
+
+  public static Converter<Byte> bytes() {
+    return ByteConverter.INSTANCE;
+  }
+
+  public static Converter<Short> shorts() {
+    return ShortConverter.INSTANCE;
+  }
+
+  public static Converter<Integer> ints() {
+    return IntConverter.INSTANCE;
+  }
+
+  public static Converter<LocalTime> times() {
+    return TimeConverter.INSTANCE;
+  }
+
+  public static Converter<Long> longs() {
+    return LongConverter.INSTANCE;
+  }
+
+  public static Converter<Float> floats() {
+    return FloatConverter.INSTANCE;
+  }
+
+  public static Converter<Double> doubles() {
+    return DoubleConverter.INSTANCE;
+  }
+
+  public static Converter<String> strings() {
+    return StringConverter.INSTANCE;
+  }
+
+  public static Converter<ByteBuffer> binary() {
+    return BytesConverter.INSTANCE;
+  }
+
+  public static Converter<UUID> uuids() {
+    return UUIDConverter.INSTANCE;
+  }
+
+  public static Converter<byte[]> fixed() {
+    return FixedConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDate> dates() {
+    return DateConverter.INSTANCE;
+  }
+
+  public static Converter<OffsetDateTime> timestampTz() {
+    return TimestampTzConverter.INSTANCE;
+  }
+
+  public static Converter<LocalDateTime> timestamp() {
+    return TimestampConverter.INSTANCE;
+  }
+
+  public static Converter<BigDecimal> decimal18(TypeDescription schema) {
+    return new Decimal18Converter(schema);
+  }
+
+  public static Converter<BigDecimal> decimal38(TypeDescription schema) {
+    return Decimal38Converter.INSTANCE;
+  }
+
+  private static class BooleanConverter implements Converter<Boolean> {
+    private static final Converter<Boolean> INSTANCE = new BooleanConverter();
+
+    @Override
+    public Class<Boolean> 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;
+      }
+    }
+  }
+
+  private static class ByteConverter implements Converter<Byte> {
+    private static final Converter<Byte> INSTANCE = new ByteConverter();
+
+    @Override
+    public Class<Byte> getJavaClass() {
+      return Byte.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class ShortConverter implements Converter<Short> {
+    private static final Converter<Short> INSTANCE = new ShortConverter();
+
+    @Override
+    public Class<Short> getJavaClass() {
+      return Short.class;
+    }
+
+    @Override
+    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;
+      }
+    }
+  }
+
+  private static class IntConverter implements Converter<Integer> {
+    private static final Converter<Integer> INSTANCE = new IntConverter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Integer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class TimeConverter implements Converter<LocalTime> {
+    private static final Converter<LocalTime> INSTANCE = new TimeConverter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+      }
+    }
+  }
+
+  private static class LongConverter implements Converter<Long> {
+    private static final Converter<Long> INSTANCE = new LongConverter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Long data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class FloatConverter implements Converter<Float> {
+    private static final Converter<Float> INSTANCE = new FloatConverter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Float data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class DoubleConverter implements Converter<Double> {
+    private static final Converter<Double> INSTANCE = new DoubleConverter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void addValue(int rowId, Double data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DoubleColumnVector) output).vector[rowId] = data;
+      }
+    }
+  }
+
+  private static class StringConverter implements Converter<String> {
+    private static final Converter<String> INSTANCE = new StringConverter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void addValue(int rowId, String data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        byte[] value = data.getBytes(StandardCharsets.UTF_8);
+        ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+      }
+    }
+  }
+
+  private static class BytesConverter implements Converter<ByteBuffer> {
+    private static final Converter<ByteBuffer> INSTANCE = new BytesConverter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void addValue(int rowId, ByteBuffer data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+      }
+    }
+  }
+
+  private static class UUIDConverter implements Converter<UUID> {
+    private static final Converter<UUID> INSTANCE = new UUIDConverter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void addValue(int rowId, UUID data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ByteBuffer buffer = ByteBuffer.allocate(16);
+        buffer.putLong(data.getMostSignificantBits());
+        buffer.putLong(data.getLeastSignificantBits());
+        ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+      }
+    }
+  }
+
+  private static class FixedConverter implements Converter<byte[]> {
+    private static final Converter<byte[]> INSTANCE = new FixedConverter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void addValue(int rowId, byte[] data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+      }
+    }
+  }
+
+  private static class DateConverter implements Converter<LocalDate> {
+    private static final Converter<LocalDate> INSTANCE = new DateConverter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDate data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+      }
+    }
+  }
+
+  private static class TimestampTzConverter implements Converter<OffsetDateTime> {
+    private static final Converter<OffsetDateTime> INSTANCE = new TimestampTzConverter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, OffsetDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class TimestampConverter implements Converter<LocalDateTime> {
+    private static final Converter<LocalDateTime> INSTANCE = new TimestampConverter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void addValue(int rowId, LocalDateTime data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        TimestampColumnVector cv = (TimestampColumnVector) output;
+        cv.setIsUTC(true);
+        cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+        cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+      }
+    }
+  }
+
+  private static class Decimal18Converter implements Converter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Converter(TypeDescription schema) {
+      this.scale = schema.getScale();
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId]
+            .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+      }
+    }
+  }
+
+  private static class Decimal38Converter implements Converter<BigDecimal> {
+    private static final Converter<BigDecimal> INSTANCE = new Decimal38Converter();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void addValue(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+      }
+    }
+  }
+
+  public static class RecordConverter implements Converter<Record> {
+    private final List<Converter> converters;
+
+    RecordConverter(List<Converter> converters) {
+      this.converters = converters;
+    }
+
+    public List<Converter> converters() {
+      return converters;
+    }
+
+    @Override
+    public Class<Record> getJavaClass() {
+      return Record.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void addValue(int rowId, Record data, ColumnVector output) {
+      if (data == null) {
+        output.noNulls = false;
+        output.isNull[rowId] = true;
+      } else {
+        output.isNull[rowId] = false;
+        StructColumnVector cv = (StructColumnVector) output;
+        for (int c = 0; c < converters.size(); ++c) {
+          converters.get(c).addValue(rowId, data.get(c, converters.get(c).getJavaClass()), cv.fields[c]);
+        }
+      }
+    }
+  }
+
+  public static class ListConverter implements Converter<List> {
+    private final Converter children;

Review comment:
       Nit: rename to `elementConverter`? Similar to what we do for `MapConverter`




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454774011



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -434,175 +144,12 @@ public void addValue(int rowId, BigDecimal data, ColumnVector output) {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
+    public void nonNullWrite(int rowId, Record data, ColumnVector output) {
+      StructColumnVector cv = (StructColumnVector) output;
+      for (int c = 0; c < writers.size(); ++c) {
+        OrcValueWriter child = writers.get(c);
+        child.write(rowId, data.get(c, child.getJavaClass()), cv.fields[c]);
       }
     }
   }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();

Review comment:
       Seems the `TypeID#javaClass`  defines the literal type, means the type to compare or serialize/deserialize. Actually, I'd prefer to  pass the `LocalDateTime` object to the comparator and do the `LocalDateTime` to `Long` conversion when comparing.  Then  we hidden the literal types between different types, and the upper layer won't to  wrap again and again.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-664079522


   @rdsr  I've refactored the SparkOrcWrier by using OrcSchemaWithTypeVisitor in [here](https://github.com/apache/iceberg/pull/1238/files#diff-5aa4840155ec70fdf7f725e122cde7b7R44),  we can see that the constructor of SparkOrcWriter will also need the two arguments:  iceberg schema and TypeDescription.  So actually,  although we could add a `Function` in here, but we actually won't use it in the newly refactored spark orc writer, that's why I say we don't need to introduce the redundant `Function` method. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458230021



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {

Review comment:
       We still need to add the parameterized types.
   
   The FAQ entry you pointed to explains why there is no class literal, like `List<String>.class`. All variants of `List` use `List.class` because there is only one concrete type at runtime. But we still want to use type parameters to be explicit about what is passed around.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454764489



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -434,175 +144,12 @@ public void addValue(int rowId, BigDecimal data, ColumnVector output) {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
+    public void nonNullWrite(int rowId, Record data, ColumnVector output) {
+      StructColumnVector cv = (StructColumnVector) output;
+      for (int c = 0; c < writers.size(); ++c) {
+        OrcValueWriter child = writers.get(c);
+        child.write(rowId, data.get(c, child.getJavaClass()), cv.fields[c]);
       }
     }
   }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();

Review comment:
       I see.  The TypeID#javaClass defines the class of iceberg desired data types(expressions as you said), the `Record`'s data type is different, we also have a class to transform between them . https://github.com/apache/iceberg/blob/68e417c3591439044b229b657a529cad0fec66e3/data/src/main/java/org/apache/iceberg/data/InternalRecordWrapper.java#L45 
   But another question is: why we need to use the different data types in `Expression` or manifest files serialization & deserialization, which produces this complexity....




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455331348



##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -65,9 +65,10 @@ public void write(InternalRow value, VectorizedRowBatch output) {
   interface Converter {
     /**
      * Take a value from the Spark data value and add it to the ORC output.
-     * @param rowId the row in the ColumnVector
+     *
+     * @param rowId  the row in the ColumnVector
      * @param column either the column number or element number
-     * @param data either an InternalRow or ArrayData
+     * @param data   either an InternalRow or ArrayData

Review comment:
       Nit: unnecessary whitespace changes.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454053131



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       For me, it seems don't have much difference with a `<?>` or not,  but  I can changed to keep symmetry as we've discussed above. 

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;

Review comment:
       It's a good idea to preserve symmetry, one question is: the current OrcValueReader is a `public` interfaces,  do the refactor seems will affect the downstream users if we don't have some `deprecated` ways.  

##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);

Review comment:
       Well, let me take a look how to handle those issues you said, thanks for your suggession, @shardulm94 . 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r457804242



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -19,590 +19,119 @@
 
 package org.apache.iceberg.data.orc;
 
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.StandardCharsets;
-import java.time.Instant;
-import java.time.LocalDate;
-import java.time.LocalDateTime;
-import java.time.LocalTime;
-import java.time.OffsetDateTime;
-import java.time.ZoneOffset;
-import java.time.temporal.ChronoUnit;
 import java.util.List;
-import java.util.Map;
-import java.util.UUID;
+import org.apache.iceberg.Schema;
 import org.apache.iceberg.data.Record;
 import org.apache.iceberg.orc.ORCSchemaUtil;
+import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor;
 import org.apache.iceberg.orc.OrcValueWriter;
-import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.types.Type;
+import org.apache.iceberg.types.Types;
 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 GenericOrcWriter implements OrcValueWriter<Record> {
-  private final Converter[] converters;
-  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
-  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
-
-  private GenericOrcWriter(TypeDescription schema) {
-    this.converters = buildConverters(schema);
-  }
-
-  public static OrcValueWriter<Record> buildWriter(TypeDescription fileSchema) {
-    return new GenericOrcWriter(fileSchema);
+  private final GenericOrcWriters.Converter converter;
+
+  private GenericOrcWriter(Schema expectedSchema, TypeDescription orcSchema) {
+    Preconditions.checkArgument(orcSchema.getCategory() == TypeDescription.Category.STRUCT,
+        "Top level must be a struct " + orcSchema);
+
+    converter = OrcSchemaWithTypeVisitor.visit(expectedSchema, orcSchema, new WriteBuilder());
+  }
+
+  public static OrcValueWriter<Record> buildWriter(Schema expectedSchema, TypeDescription fileSchema) {
+    return new GenericOrcWriter(expectedSchema, fileSchema);
+  }
+
+  private static class WriteBuilder extends OrcSchemaWithTypeVisitor<GenericOrcWriters.Converter> {
+    private WriteBuilder() {
+    }
+
+    public GenericOrcWriters.Converter record(Types.StructType iStruct, TypeDescription record,
+                                              List<String> names, List<GenericOrcWriters.Converter> fields) {
+      return new GenericOrcWriters.RecordConverter(fields);
+    }
+
+    public GenericOrcWriters.Converter list(Types.ListType iList, TypeDescription array,
+                                            GenericOrcWriters.Converter element) {
+      return new GenericOrcWriters.ListConverter(element);
+    }
+
+    public GenericOrcWriters.Converter map(Types.MapType iMap, TypeDescription map,
+                                           GenericOrcWriters.Converter key, GenericOrcWriters.Converter value) {
+      return new GenericOrcWriters.MapConverter(key, value);
+    }
+
+    public GenericOrcWriters.Converter primitive(Type.PrimitiveType iPrimitive, TypeDescription schema) {
+      switch (schema.getCategory()) {
+        case BOOLEAN:
+          return GenericOrcWriters.booleans();
+        case BYTE:
+          return GenericOrcWriters.bytes();
+        case SHORT:
+          return GenericOrcWriters.shorts();
+        case DATE:
+          return GenericOrcWriters.dates();
+        case INT:
+          return GenericOrcWriters.ints();
+        case LONG:
+          String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
+              ORCSchemaUtil.LongType.valueOf(longAttributeValue);
+          switch (longType) {
+            case TIME:
+              return GenericOrcWriters.times();
+            case LONG:
+              return GenericOrcWriters.longs();
+            default:
+              throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
+          }
+        case FLOAT:
+          return GenericOrcWriters.floats();
+        case DOUBLE:
+          return GenericOrcWriters.doubles();
+        case BINARY:
+          String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
+          ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
+              ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
+          switch (binaryType) {
+            case UUID:
+              return GenericOrcWriters.uuids();
+            case FIXED:
+              return GenericOrcWriters.fixed();
+            case BINARY:
+              return GenericOrcWriters.binary();
+            default:
+              throw new IllegalStateException("Unhandled Binary type found in ORC type attribute: " + binaryType);
+          }
+        case STRING:
+        case CHAR:
+        case VARCHAR:
+          return GenericOrcWriters.strings();
+        case DECIMAL:
+          return schema.getPrecision() <= 18 ? GenericOrcWriters.decimal18(schema) :
+              GenericOrcWriters.decimal38(schema);
+        case TIMESTAMP:
+          return GenericOrcWriters.timestamp();
+        case TIMESTAMP_INSTANT:
+          return GenericOrcWriters.timestampTz();
+      }
+      throw new IllegalArgumentException("Unhandled type " + schema);
+    }
   }
 
   @SuppressWarnings("unchecked")

Review comment:
       Here I did not change to use `OrcValueWriter<?>` because  if we do then we have the following to write child:
   ```java
   for (int c = 0; c < writers.size(); ++c) {
         OrcValueWriter<?> child = writers.get(c);
         child.write(row, value.get(c, child.getJavaClass()), output.cols[c]);
   }
   ```
   
   The `value` is a `StructLike` and the  `get` in `StructLike` is `<T> T get(int pos, Class<T> javaClass)`, while child.getJavaClass is a class like `OrcValueWriter<?>.class`, it will throw the compile error: 
   ```
   Incompatible types. Required capture of ? but 'get' was inferred to T: no instance(s) of of type variables(s) exist so that capture of ? conforms to capture of ?
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459250716



##########
File path: data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java
##########
@@ -180,7 +180,7 @@ public void createOrcInputFile(List<Record> records) throws IOException {
     OutputFile outFile = Files.localOutput(ORC_FILE);
     try (FileAppender<Record> appender = ORC.write(outFile)
         .schema(FILE_SCHEMA)
-        .createWriterFunc(GenericOrcWriter::buildWriter)
+        .createWriterFunc(typeDesc -> GenericOrcWriter.buildWriter(FILE_SCHEMA, typeDesc))

Review comment:
       Would it make more sense to add `createWriterFunc` method that accepts `BiFunction<Schema, TypeDescription>` instead of replacing the existing one? Replacing the existing `createWriterFunc`  causes changes in the files?
   ```
   TestSparkOrcReadMetadataColumns.java
   TestSparkOrcReader.java
   TestOrcWrite.java
   SparkAppenderFactory.java
   ```




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455328952



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriters.java
##########
@@ -0,0 +1,418 @@
+/*
+ * 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.data.orc;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.OffsetDateTime;
+import java.time.ZoneOffset;
+import java.time.temporal.ChronoUnit;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import org.apache.iceberg.orc.OrcValueWriter;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+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.TimestampColumnVector;
+
+public class GenericOrcWriters {
+  private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC);
+  private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate();
+
+  private GenericOrcWriters() {
+  }
+
+  public static OrcValueWriter<Boolean> booleans() {
+    return BooleanWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Integer> ints() {
+    return IntWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalTime> times() {
+    return TimeWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Long> longs() {
+    return LongWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Float> floats() {
+    return FloatWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<Double> doubles() {
+    return DoubleWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<String> strings() {
+    return StringWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<ByteBuffer> byteBuffers() {
+    return ByteBufferWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<UUID> uuids() {
+    return UUIDWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<byte[]> fixed() {
+    return FixedWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDate> dates() {
+    return DateWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<OffsetDateTime> timestampTz() {
+    return TimestampTzWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<LocalDateTime> timestamp() {
+    return TimestampWriter.INSTANCE;
+  }
+
+  public static OrcValueWriter<BigDecimal> decimal(int scala, int precision) {
+    if (precision <= 18) {
+      return new Decimal18Writer(scala);
+    } else {
+      return Decimal38Writer.INSTANCE;
+    }
+  }
+
+  public static OrcValueWriter<List> list(OrcValueWriter element) {
+    return new ListWriter(element);
+  }
+
+  public static OrcValueWriter<Map> map(OrcValueWriter key, OrcValueWriter value) {
+    return new MapWriter(key, value);
+  }
+
+  private static class BooleanWriter implements OrcValueWriter<Boolean> {
+    private static final OrcValueWriter<Boolean> INSTANCE = new BooleanWriter();
+
+    @Override
+    public Class<Boolean> getJavaClass() {
+      return Boolean.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Boolean data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data ? 1 : 0;
+    }
+  }
+
+  private static class IntWriter implements OrcValueWriter<Integer> {
+    private static final OrcValueWriter<Integer> INSTANCE = new IntWriter();
+
+    @Override
+    public Class<Integer> getJavaClass() {
+      return Integer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Integer data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class TimeWriter implements OrcValueWriter<LocalTime> {
+    private static final OrcValueWriter<LocalTime> INSTANCE = new TimeWriter();
+
+    @Override
+    public Class<LocalTime> getJavaClass() {
+      return LocalTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalTime data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data.toNanoOfDay() / 1_000;
+    }
+  }
+
+  private static class LongWriter implements OrcValueWriter<Long> {
+    private static final OrcValueWriter<Long> INSTANCE = new LongWriter();
+
+    @Override
+    public Class<Long> getJavaClass() {
+      return Long.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Long data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class FloatWriter implements OrcValueWriter<Float> {
+    private static final OrcValueWriter<Float> INSTANCE = new FloatWriter();
+
+    @Override
+    public Class<Float> getJavaClass() {
+      return Float.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Float data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class DoubleWriter implements OrcValueWriter<Double> {
+    private static final OrcValueWriter<Double> INSTANCE = new DoubleWriter();
+
+    @Override
+    public Class<Double> getJavaClass() {
+      return Double.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, Double data, ColumnVector output) {
+      ((DoubleColumnVector) output).vector[rowId] = data;
+    }
+  }
+
+  private static class StringWriter implements OrcValueWriter<String> {
+    private static final OrcValueWriter<String> INSTANCE = new StringWriter();
+
+    @Override
+    public Class<String> getJavaClass() {
+      return String.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, String data, ColumnVector output) {
+      byte[] value = data.getBytes(StandardCharsets.UTF_8);
+      ((BytesColumnVector) output).setRef(rowId, value, 0, value.length);
+    }
+  }
+
+  private static class ByteBufferWriter implements OrcValueWriter<ByteBuffer> {
+    private static final OrcValueWriter<ByteBuffer> INSTANCE = new ByteBufferWriter();
+
+    @Override
+    public Class<ByteBuffer> getJavaClass() {
+      return ByteBuffer.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, ByteBuffer data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data.array(), 0, data.array().length);
+    }
+  }
+
+  private static class UUIDWriter implements OrcValueWriter<UUID> {
+    private static final OrcValueWriter<UUID> INSTANCE = new UUIDWriter();
+
+    @Override
+    public Class<UUID> getJavaClass() {
+      return UUID.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, UUID data, ColumnVector output) {
+      ByteBuffer buffer = ByteBuffer.allocate(16);
+      buffer.putLong(data.getMostSignificantBits());
+      buffer.putLong(data.getLeastSignificantBits());
+      ((BytesColumnVector) output).setRef(rowId, buffer.array(), 0, buffer.array().length);
+    }
+  }
+
+  private static class FixedWriter implements OrcValueWriter<byte[]> {
+    private static final OrcValueWriter<byte[]> INSTANCE = new FixedWriter();
+
+    @Override
+    public Class<byte[]> getJavaClass() {
+      return byte[].class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, byte[] data, ColumnVector output) {
+      ((BytesColumnVector) output).setRef(rowId, data, 0, data.length);
+    }
+  }
+
+  private static class DateWriter implements OrcValueWriter<LocalDate> {
+    private static final OrcValueWriter<LocalDate> INSTANCE = new DateWriter();
+
+    @Override
+    public Class<LocalDate> getJavaClass() {
+      return LocalDate.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDate data, ColumnVector output) {
+      ((LongColumnVector) output).vector[rowId] = ChronoUnit.DAYS.between(EPOCH_DAY, data);
+    }
+  }
+
+  private static class TimestampTzWriter implements OrcValueWriter<OffsetDateTime> {
+    private static final OrcValueWriter<OffsetDateTime> INSTANCE = new TimestampTzWriter();
+
+    @Override
+    public Class<OffsetDateTime> getJavaClass() {
+      return OffsetDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, OffsetDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.time[rowId] = data.toInstant().toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class TimestampWriter implements OrcValueWriter<LocalDateTime> {
+    private static final OrcValueWriter<LocalDateTime> INSTANCE = new TimestampWriter();
+
+    @Override
+    public Class<LocalDateTime> getJavaClass() {
+      return LocalDateTime.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, LocalDateTime data, ColumnVector output) {
+      TimestampColumnVector cv = (TimestampColumnVector) output;
+      cv.setIsUTC(true);
+      cv.time[rowId] = data.toInstant(ZoneOffset.UTC).toEpochMilli(); // millis
+      cv.nanos[rowId] = (data.getNano() / 1_000) * 1_000; // truncate nanos to only keep microsecond precision
+    }
+  }
+
+  private static class Decimal18Writer implements OrcValueWriter<BigDecimal> {
+    private final int scale;
+
+    Decimal18Writer(int scale) {
+      this.scale = scale;
+    }
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId]
+          .setFromLongAndScale(data.unscaledValue().longValueExact(), scale);
+    }
+  }
+
+  private static class Decimal38Writer implements OrcValueWriter<BigDecimal> {
+    private static final OrcValueWriter<BigDecimal> INSTANCE = new Decimal38Writer();
+
+    @Override
+    public Class<BigDecimal> getJavaClass() {
+      return BigDecimal.class;
+    }
+
+    @Override
+    public void nonNullWrite(int rowId, BigDecimal data, ColumnVector output) {
+      // TODO: validate precision and scale from schema
+      ((DecimalColumnVector) output).vector[rowId].set(HiveDecimal.create(data, false));
+    }
+  }
+
+  private static class ListWriter implements OrcValueWriter<List> {
+    private final OrcValueWriter element;
+
+    ListWriter(OrcValueWriter element) {
+      this.element = element;
+    }
+
+    @Override
+    public Class<List> getJavaClass() {
+      return List.class;
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void nonNullWrite(int rowId, List data, ColumnVector output) {
+      List<Object> value = (List<Object>) data;
+      ListColumnVector cv = (ListColumnVector) output;
+      // record the length and start of the list elements
+      cv.lengths[rowId] = value.size();
+      cv.offsets[rowId] = cv.childCount;
+      cv.childCount += cv.lengths[rowId];
+      // make sure the child is big enough
+      cv.child.ensureSize(cv.childCount, true);
+      // Add each element
+      for (int e = 0; e < cv.lengths[rowId]; ++e) {
+        element.write((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
+      }
+    }
+  }
+
+  private static class MapWriter implements OrcValueWriter<Map> {

Review comment:
       Missing parameter types.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdsr commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdsr commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459628689



##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -42,7 +42,7 @@
  * This class acts as an adaptor from an OrcFileAppender to a
  * FileAppender&lt;InternalRow&gt;.
  */
-public class SparkOrcWriter implements OrcValueWriter<InternalRow> {
+public class SparkOrcWriter implements OrcRowWriter<InternalRow> {
 
   private final Converter[] converters;

Review comment:
       Thankyou!




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r459274527



##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -42,7 +42,7 @@
  * This class acts as an adaptor from an OrcFileAppender to a
  * FileAppender&lt;InternalRow&gt;.
  */
-public class SparkOrcWriter implements OrcValueWriter<InternalRow> {
+public class SparkOrcWriter implements OrcRowWriter<InternalRow> {
 
   private final Converter[] converters;

Review comment:
       Yeah, that's exactly what I plan to do.  I've filed an issue to address this thing. https://github.com/apache/iceberg/issues/1236.   Thanks for bringing it up. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458510725



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java
##########
@@ -19,20 +19,28 @@
 
 package org.apache.iceberg.orc;
 
-import java.io.IOException;
-import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
 
-/**
- * Write data value of a schema.
- */
 public interface OrcValueWriter<T> {
 
+  Class<T> getJavaClass();

Review comment:
       It's used for reading the field from `Record` and casting the value to target class , see here: https://github.com/apache/iceberg/pull/1197/files#diff-69c0f1e45966d2eb49a315fe32734cf5R125. 

##########
File path: data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java
##########
@@ -180,7 +180,7 @@ public void createOrcInputFile(List<Record> records) throws IOException {
     OutputFile outFile = Files.localOutput(ORC_FILE);
     try (FileAppender<Record> appender = ORC.write(outFile)
         .schema(FILE_SCHEMA)
-        .createWriterFunc(GenericOrcWriter::buildWriter)
+        .createWriterFunc(typeDesc -> GenericOrcWriter.buildWriter(FILE_SCHEMA, typeDesc))

Review comment:
       OK, that sounds good to me. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] Simon0806 commented on pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
Simon0806 commented on pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#issuecomment-665457977


   Can we abstract a BaseOrcWriter in the future ? Then make GenericOrcWriter, FlinkOrcWriter, SparkOrcWriter extends 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:
users@infra.apache.org



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


[GitHub] [iceberg] openinx commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
openinx commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r454285996



##########
File path: data/src/main/java/org/apache/iceberg/data/orc/GenericOrcWriter.java
##########
@@ -434,175 +144,12 @@ public void addValue(int rowId, BigDecimal data, ColumnVector output) {
 
     @Override
     @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Record data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        StructColumnVector cv = (StructColumnVector) output;
-        for (int c = 0; c < children.length; ++c) {
-          children[c].addValue(rowId, data.get(c, children[c].getJavaClass()), cv.fields[c]);
-        }
-      }
-    }
-  }
-
-  static class ListConverter implements Converter<List> {
-    private final Converter children;
-
-    ListConverter(TypeDescription schema) {
-      this.children = buildConverter(schema.getChildren().get(0));
-    }
-
-    @Override
-    public Class<List> getJavaClass() {
-      return List.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, List data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        List<Object> value = (List<Object>) data;
-        ListColumnVector cv = (ListColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = value.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.child.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          children.addValue((int) (e + cv.offsets[rowId]), value.get(e), cv.child);
-        }
+    public void nonNullWrite(int rowId, Record data, ColumnVector output) {
+      StructColumnVector cv = (StructColumnVector) output;
+      for (int c = 0; c < writers.size(); ++c) {
+        OrcValueWriter child = writers.get(c);
+        child.write(rowId, data.get(c, child.getJavaClass()), cv.fields[c]);
       }
     }
   }
-
-  static class MapConverter implements Converter<Map> {
-    private final Converter keyConverter;
-    private final Converter valueConverter;
-
-    MapConverter(TypeDescription schema) {
-      this.keyConverter = buildConverter(schema.getChildren().get(0));
-      this.valueConverter = buildConverter(schema.getChildren().get(1));
-    }
-
-    @Override
-    public Class<Map> getJavaClass() {
-      return Map.class;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public void addValue(int rowId, Map data, ColumnVector output) {
-      if (data == null) {
-        output.noNulls = false;
-        output.isNull[rowId] = true;
-      } else {
-        output.isNull[rowId] = false;
-        Map<Object, Object> map = (Map<Object, Object>) data;
-        List<Object> keys = Lists.newArrayListWithExpectedSize(map.size());
-        List<Object> values = Lists.newArrayListWithExpectedSize(map.size());
-        for (Map.Entry<?, ?> entry : map.entrySet()) {
-          keys.add(entry.getKey());
-          values.add(entry.getValue());
-        }
-        MapColumnVector cv = (MapColumnVector) output;
-        // record the length and start of the list elements
-        cv.lengths[rowId] = map.size();
-        cv.offsets[rowId] = cv.childCount;
-        cv.childCount += cv.lengths[rowId];
-        // make sure the child is big enough
-        cv.keys.ensureSize(cv.childCount, true);
-        cv.values.ensureSize(cv.childCount, true);
-        // Add each element
-        for (int e = 0; e < cv.lengths[rowId]; ++e) {
-          int pos = (int) (e + cv.offsets[rowId]);
-          keyConverter.addValue(pos, keys.get(e), cv.keys);
-          valueConverter.addValue(pos, values.get(e), cv.values);
-        }
-      }
-    }
-  }
-
-  private static Converter buildConverter(TypeDescription schema) {
-    switch (schema.getCategory()) {
-      case BOOLEAN:
-        return new BooleanConverter();
-      case BYTE:
-        return new ByteConverter();
-      case SHORT:
-        return new ShortConverter();
-      case DATE:
-        return new DateConverter();
-      case INT:
-        return new IntConverter();
-      case LONG:
-        String longAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_LONG_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.LongType longType = longAttributeValue == null ? ORCSchemaUtil.LongType.LONG :
-            ORCSchemaUtil.LongType.valueOf(longAttributeValue);
-        switch (longType) {
-          case TIME:
-            return new TimeConverter();
-          case LONG:
-            return new LongConverter();
-          default:
-            throw new IllegalStateException("Unhandled Long type found in ORC type attribute: " + longType);
-        }
-      case FLOAT:
-        return new FloatConverter();
-      case DOUBLE:
-        return new DoubleConverter();
-      case BINARY:
-        String binaryAttributeValue = schema.getAttributeValue(ORCSchemaUtil.ICEBERG_BINARY_TYPE_ATTRIBUTE);
-        ORCSchemaUtil.BinaryType binaryType = binaryAttributeValue == null ? ORCSchemaUtil.BinaryType.BINARY :
-            ORCSchemaUtil.BinaryType.valueOf(binaryAttributeValue);
-        switch (binaryType) {
-          case UUID:
-            return new UUIDConverter();
-          case FIXED:
-            return new FixedConverter();

Review comment:
       Is there problem here ? I mean the iceberg  FIXED type is mapping to `ByteBuffer` class and the `FixedConverter`  will write a `byte[]` instance to the `ColumnVector`,  I guess the unit tests did not expose this issue because the `RandomDataGenerator` are producing a `ByteBuffer` instance and we did not use the TypeID#javaClass(). The `BINARY` & `TIME` have similar issues I think.    @rdsr @shardulm94  FYI. 




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455330224



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java
##########
@@ -51,12 +51,12 @@
   private final OutputFile file;
   private final Writer writer;
   private final VectorizedRowBatch batch;
-  private final OrcValueWriter<D> valueWriter;
+  private final OrcRowWriter<D> valueWriter;

Review comment:
       What is the purpose of changing `OrcValueWriter` to `OrcRowWriter`? It seems like this isn't really necessary for the purpose of this commit and causes quite a few changes. Is there value in doing this?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r458234776



##########
File path: orc/src/main/java/org/apache/iceberg/orc/OrcValueWriter.java
##########
@@ -19,20 +19,28 @@
 
 package org.apache.iceberg.orc;
 
-import java.io.IOException;
-import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch;
+import org.apache.orc.storage.ql.exec.vector.ColumnVector;
 
-/**
- * Write data value of a schema.
- */
 public interface OrcValueWriter<T> {
 
+  Class<T> getJavaClass();

Review comment:
       What will this be used for? I don't see anything calling it in this commit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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


[GitHub] [iceberg] rdblue commented on a change in pull request #1197: Refactor the GenericOrcWriter by using OrcSchemaWithTypeVisitor#visit

Posted by GitBox <gi...@apache.org>.
rdblue commented on a change in pull request #1197:
URL: https://github.com/apache/iceberg/pull/1197#discussion_r455331348



##########
File path: spark/src/main/java/org/apache/iceberg/spark/data/SparkOrcWriter.java
##########
@@ -65,9 +65,10 @@ public void write(InternalRow value, VectorizedRowBatch output) {
   interface Converter {
     /**
      * Take a value from the Spark data value and add it to the ORC output.
-     * @param rowId the row in the ColumnVector
+     *
+     * @param rowId  the row in the ColumnVector
      * @param column either the column number or element number
-     * @param data either an InternalRow or ArrayData
+     * @param data   either an InternalRow or ArrayData

Review comment:
       Nit: unnecessary whitespace change.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



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