lidavidm commented on code in PR #638:
URL: https://github.com/apache/arrow-java/pull/638#discussion_r1980767717


##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStructProducer.java:
##########
@@ -0,0 +1,73 @@
+/*
+ * 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.arrow.adapter.avro.producers;
+
+import java.io.IOException;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer which produces nested record type values to avro encoder. Read the 
data from {@link
+ * org.apache.arrow.vector.complex.StructVector}.
+ */
+public class AvroStructProducer extends BaseAvroProducer<StructVector> {
+
+  private final Producer<? extends FieldVector>[] delegates;
+
+  /** Instantiate a AvroStructProducer. */
+  public AvroStructProducer(StructVector vector, Producer<? extends 
FieldVector>[] delegates) {
+    super(vector);
+    this.delegates = delegates;
+  }
+
+  @Override
+  public void produce(Encoder encoder) throws IOException {
+
+    for (Producer<?> delegate : delegates) {
+      delegate.produce(encoder);
+    }
+
+    currentIndex++;
+  }
+
+  @Override
+  public void skipNull() {
+    for (Producer<?> delegate : delegates) {
+      delegate.skipNull();
+    }
+    super.skipNull();
+  }
+
+  @Override
+  public void setPosition(int index) {
+    for (Producer<?> delegate : delegates) {
+      delegate.setPosition(index);
+    }
+    super.setPosition(index);
+  }
+
+  @Override
+  @SuppressWarnings("unchecked")
+  public boolean resetValueVector(StructVector vector) {
+    for (int i = 0; i < delegates.length; i++) {
+      Producer<FieldVector> delegate = (Producer<FieldVector>) delegates[i];
+      delegate.resetValueVector(vector.getChildrenFromFields().get(i));

Review Comment:
   Should the return value here be accounted for?



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java:
##########
@@ -0,0 +1,729 @@
+/*
+ * 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.arrow.adapter.avro;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.arrow.adapter.avro.producers.AvroBigIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroBooleanProducer;
+import org.apache.arrow.adapter.avro.producers.AvroBytesProducer;
+import org.apache.arrow.adapter.avro.producers.AvroDenseUnionProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFixedSizeBinaryProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFixedSizeListProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat2Producer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat4Producer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat8Producer;
+import org.apache.arrow.adapter.avro.producers.AvroIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroListProducer;
+import org.apache.arrow.adapter.avro.producers.AvroMapProducer;
+import org.apache.arrow.adapter.avro.producers.AvroNullProducer;
+import org.apache.arrow.adapter.avro.producers.AvroNullableProducer;
+import org.apache.arrow.adapter.avro.producers.AvroSmallIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroStringProducer;
+import org.apache.arrow.adapter.avro.producers.AvroStructProducer;
+import org.apache.arrow.adapter.avro.producers.AvroTinyIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroUint1Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint2Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint4Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint8Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUnionProducer;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.adapter.avro.producers.CompositeAvroProducer;
+import org.apache.arrow.adapter.avro.producers.Producer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDateDayProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDateMilliProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDecimal256Producer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDecimalProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeMicroProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeMilliProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeNanoProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeSecProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMicroProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMicroTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMilliProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMilliTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampNanoProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampNanoTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampSecProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampSecTzProducer;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float2Vector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.NullVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampNanoVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampSecVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.UnionVector;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+
+public class ArrowToAvroUtils {
+
+  public static final String GENERIC_RECORD_TYPE_NAME = "GenericRecord";
+
+  /**
+   * Create an Avro record schema for a given list of Arrow fields.
+   *
+   * <p>This method currently performs following type mapping for Avro data 
types to corresponding
+   * Arrow data types.
+   *
+   * <ul>
+   *   <li>ArrowType.Null --> NULL

Review Comment:
   not to nitpick the formatting but a table might be better? (also does 
Javadoc allow unclosed tags like this?)



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/BaseTimestampTzProducer.java:
##########
@@ -0,0 +1,58 @@
+/*
+ * 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.arrow.adapter.avro.producers.logical;
+
+import java.io.IOException;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.avro.io.Encoder;
+
+abstract class BaseTimestampTzProducer<T extends TimeStampVector>
+    extends BaseAvroProducer<TimeStampVector> {
+
+  // Convert TZ values to UTC to encode Avro timestamp types

Review Comment:
   Arrow stores zoned timestamps in UTC already. The time zone associated with 
the type is purely used for display. So there should be no need for conversion.
   
   
https://github.com/apache/arrow/blob/601fde9033b5def7e667ffec70c0acacd1b9ad50/format/Schema.fbs#L284-L302



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint4Producer.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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.arrow.adapter.avro.producers;
+
+import java.io.IOException;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces long values from a {@link UInt4Vector}, writes data 
to an avro encoder.
+ */
+public class AvroUint4Producer extends BaseAvroProducer<UInt4Vector> {
+
+  /** Instantiate an AvroUint4Producer. */
+  public AvroUint4Producer(UInt4Vector vector) {
+    super(vector);
+  }
+
+  @Override
+  public void produce(Encoder encoder) throws IOException {
+    int unsigned = vector.getDataBuffer().getInt(currentIndex * (long) 
UInt4Vector.TYPE_WIDTH);
+    long signed = unsigned & 0xffffffffL;

Review Comment:
   Is this 
https://docs.oracle.com/javase/8/docs/api/java/lang/Integer.html#toUnsignedLong-int-
 ?



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/Producer.java:
##########
@@ -0,0 +1,53 @@
+/*
+ * 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.arrow.adapter.avro.producers;
+
+import java.io.IOException;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Interface that is used to produce values to avro encoder.
+ *
+ * @param <T> The vector within producer or its delegate, used for partially 
produce purpose.
+ */
+public interface Producer<T extends FieldVector> {
+
+  /**
+   * Produce a specific type value from the vector and write it to avro 
encoder.
+   *
+   * @param encoder avro encoder to write data
+   * @throws IOException on error
+   */
+  void produce(Encoder encoder) throws IOException;
+
+  /** Skip null value in the vector by setting reader position + 1. */
+  void skipNull();
+
+  /** Set the position to read value from vector. */
+  void setPosition(int index);
+
+  /**
+   * Reset the vector within producer.
+   *
+   * @return true if reset is successful, false if reset is not needed.
+   */
+  boolean resetValueVector(T vector);

Review Comment:
   Hmm, do any of the implementations ever return `false`?



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMilliProducer.java:
##########
@@ -0,0 +1,34 @@
+/*
+ * 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.arrow.adapter.avro.producers.logical;
+
+import org.apache.arrow.adapter.avro.producers.AvroIntProducer;
+import org.apache.arrow.vector.TimeMilliVector;
+
+/**
+ * Producer that produces time (milliseconds) values from a {@link 
TimeMilliVector}, writes data to
+ * an Avro encoder.
+ */
+public class AvroTimeMilliProducer extends AvroIntProducer {
+
+  // Time in milliseconds stored as integer, matches Avro time-millis type

Review Comment:
   Hmm, Arrow timestamp type is always 8 bytes, so won't this suffer a range 
issue?



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/Producer.java:
##########
@@ -0,0 +1,62 @@
+/*
+ * 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.arrow.adapter.avro.producers;
+
+import org.apache.arrow.vector.FieldVector;
+import org.apache.avro.io.Encoder;
+
+import java.io.IOException;
+
+/**
+ * Interface that is used to produce values to avro encoder.
+ *
+ * @param <T> The vector within producer or its delegate, used for partially 
produce purpose.
+ */
+public interface Producer<T extends FieldVector> extends AutoCloseable {
+
+  /**
+   * Produce a specific type value from the vector and write it to avro 
encoder.
+   *
+   * @param encoder avro encoder to write data
+   * @throws IOException on error
+   */
+  void produce(Encoder encoder) throws IOException;
+
+  /** Skip null value in the vector by setting reader position + 1. */
+  void skipNull();
+
+  /** Set the position to read value from vector. */
+  void setPosition(int index);

Review Comment:
   Ok, let's leave it then. (We really need to fix this more generally...)



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint2Producer.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.arrow.adapter.avro.producers;
+
+import java.io.IOException;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.avro.io.Encoder;
+
+/** Producer that produces int values from a {@link UInt2Vector}, writes data 
to an avro encoder. */
+public class AvroUint2Producer extends BaseAvroProducer<UInt2Vector> {
+
+  /** Instantiate an AvroUint2Producer. */
+  public AvroUint2Producer(UInt2Vector vector) {
+    super(vector);
+  }
+
+  @Override
+  public void produce(Encoder encoder) throws IOException {
+    short unsigned = vector.getDataBuffer().getShort(currentIndex * (long) 
UInt2Vector.TYPE_WIDTH);
+    int signed = unsigned & 0xffff;

Review Comment:
   Same here, is Short#toUnsignedInt applicable?



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java:
##########
@@ -0,0 +1,729 @@
+/*
+ * 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.arrow.adapter.avro;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.arrow.adapter.avro.producers.AvroBigIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroBooleanProducer;
+import org.apache.arrow.adapter.avro.producers.AvroBytesProducer;
+import org.apache.arrow.adapter.avro.producers.AvroDenseUnionProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFixedSizeBinaryProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFixedSizeListProducer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat2Producer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat4Producer;
+import org.apache.arrow.adapter.avro.producers.AvroFloat8Producer;
+import org.apache.arrow.adapter.avro.producers.AvroIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroListProducer;
+import org.apache.arrow.adapter.avro.producers.AvroMapProducer;
+import org.apache.arrow.adapter.avro.producers.AvroNullProducer;
+import org.apache.arrow.adapter.avro.producers.AvroNullableProducer;
+import org.apache.arrow.adapter.avro.producers.AvroSmallIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroStringProducer;
+import org.apache.arrow.adapter.avro.producers.AvroStructProducer;
+import org.apache.arrow.adapter.avro.producers.AvroTinyIntProducer;
+import org.apache.arrow.adapter.avro.producers.AvroUint1Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint2Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint4Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUint8Producer;
+import org.apache.arrow.adapter.avro.producers.AvroUnionProducer;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.adapter.avro.producers.CompositeAvroProducer;
+import org.apache.arrow.adapter.avro.producers.Producer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDateDayProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDateMilliProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDecimal256Producer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroDecimalProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeMicroProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeMilliProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeNanoProducer;
+import org.apache.arrow.adapter.avro.producers.logical.AvroTimeSecProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMicroProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMicroTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMilliProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampMilliTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampNanoProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampNanoTzProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampSecProducer;
+import 
org.apache.arrow.adapter.avro.producers.logical.AvroTimestampSecTzProducer;
+import org.apache.arrow.util.Preconditions;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.arrow.vector.BitVector;
+import org.apache.arrow.vector.DateDayVector;
+import org.apache.arrow.vector.DateMilliVector;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.arrow.vector.Float2Vector;
+import org.apache.arrow.vector.Float4Vector;
+import org.apache.arrow.vector.Float8Vector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.NullVector;
+import org.apache.arrow.vector.SmallIntVector;
+import org.apache.arrow.vector.TimeMicroVector;
+import org.apache.arrow.vector.TimeMilliVector;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.arrow.vector.TimeSecVector;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+import org.apache.arrow.vector.TimeStampMicroVector;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+import org.apache.arrow.vector.TimeStampMilliVector;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+import org.apache.arrow.vector.TimeStampNanoVector;
+import org.apache.arrow.vector.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampSecVector;
+import org.apache.arrow.vector.TinyIntVector;
+import org.apache.arrow.vector.UInt1Vector;
+import org.apache.arrow.vector.UInt2Vector;
+import org.apache.arrow.vector.UInt4Vector;
+import org.apache.arrow.vector.UInt8Vector;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.complex.DenseUnionVector;
+import org.apache.arrow.vector.complex.FixedSizeListVector;
+import org.apache.arrow.vector.complex.ListVector;
+import org.apache.arrow.vector.complex.MapVector;
+import org.apache.arrow.vector.complex.StructVector;
+import org.apache.arrow.vector.complex.UnionVector;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.Types;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.avro.Schema;
+import org.apache.avro.SchemaBuilder;
+
+public class ArrowToAvroUtils {
+
+  public static final String GENERIC_RECORD_TYPE_NAME = "GenericRecord";
+
+  /**
+   * Create an Avro record schema for a given list of Arrow fields.
+   *
+   * <p>This method currently performs following type mapping for Avro data 
types to corresponding
+   * Arrow data types.
+   *
+   * <ul>
+   *   <li>ArrowType.Null --> NULL
+   *   <li>ArrowType.Bool --> BOOLEAN
+   *   <li>ArrowType.Int(64 bit, unsigned 32 bit) --> LONG
+   *   <li>ArrowType.Int(signed 32 bit, < 32 bit) --> INT
+   *   <li>ArrowType.FloatingPoint(double) --> DOUBLE
+   *   <li>ArrowType.FloatingPoint(single, half) --> FLOAT
+   *   <li>ArrowType.Utf8 --> STRING
+   *   <li>ArrowType.LargeUtf8 --> STRING
+   *   <li>ArrowType.Binary --> BYTES
+   *   <li>ArrowType.LargeBinary --> BYTES
+   *   <li>ArrowType.FixedSizeBinary --> FIXED
+   *   <li>ArrowType.Decimal --> decimal (FIXED)
+   *   <li>ArrowType.Date --> date (INT)
+   *   <li>ArrowType.Time (MILLI) --> time-millis (INT)
+   *   <li>ArrowType.Time (SEC | MICRO | NANO) --> time-micros (LONG)
+   *   <li>ArrowType.Timestamp (NANOSECONDS, TZ != NULL) --> time-nanos (LONG)
+   *   <li>ArrowType.Timestamp (MICROSECONDS, TZ != NULL) --> time-micros 
(LONG)
+   *   <li>ArrowType.Timestamp (MILLISECONDS | SECONDS, TZ != NULL) --> 
time-millis (LONG)
+   *   <li>ArrowType.Timestamp (NANOSECONDS, TZ == NULL) --> local-time-nanos 
(LONG)
+   *   <li>ArrowType.Timestamp (MICROSECONDS, TZ == NULL) --> 
local-time-micros (LONG)
+   *   <li>ArrowType.Timestamp (MILLISECONDS | SECONDS, TZ == NULL) --> 
local-time-millis (LONG)
+   *   <li>ArrowType.Duration --> duration (FIXED)
+   *   <li>ArrowType.Interval --> duration (FIXED)
+   *   <li>ArrowType.Struct --> record
+   *   <li>ArrowType.List --> array
+   *   <li>ArrowType.LargeList --> array
+   *   <li>ArrowType.FixedSizeList --> array
+   *   <li>ArrowType.Map --> map
+   *   <li>ArrowType.Union --> union
+   * </ul>
+   *
+   * <p>Nullable fields are represented as a union of [null | base-type]. 
Special treatment is given
+   * to nullability of unions - a union is considered nullable if the union 
field is nullable or any

Review Comment:
   FWIW, I think nullable unions aren't semantically valid - unions have no 
validity bitmap in Arrow. The only way to represent this is with a nullable 
child field



##########
adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeNanoProducer.java:
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.arrow.adapter.avro.producers.logical;
+
+import java.io.IOException;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.vector.TimeNanoVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that converts nanoseconds from a {@link TimeNanoVector} and 
produces time (microseconds)
+ * values, writes data to an Avro encoder.
+ */
+public class AvroTimeNanoProducer extends BaseAvroProducer<TimeNanoVector> {
+
+  // Convert nanoseconds to microseconds for Avro time-micros (LONG) type
+  // Range is 1000 times less than for microseconds, so the type will fit 
(with loss of precision)
+
+  private static final long NANOS_PER_MICRO = 1000;
+
+  public AvroTimeNanoProducer(TimeNanoVector vector) {
+    super(vector);
+  }
+
+  @Override
+  public void produce(Encoder encoder) throws IOException {
+    long nanos = vector.getDataBuffer().getLong(currentIndex * (long) 
TimeNanoVector.TYPE_WIDTH);
+    long micros = nanos / NANOS_PER_MICRO;

Review Comment:
   Maybe we should throw on truncation to be safe?



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

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to