diff --git a/adapter/avro/src/main/java/module-info.java b/adapter/avro/src/main/java/module-info.java
index 5c6204be6..fee6c7219 100644
--- a/adapter/avro/src/main/java/module-info.java
+++ b/adapter/avro/src/main/java/module-info.java
@@ -18,6 +18,8 @@
module org.apache.arrow.adapter.avro {
exports org.apache.arrow.adapter.avro.consumers;
exports org.apache.arrow.adapter.avro.consumers.logical;
+ exports org.apache.arrow.adapter.avro.producers;
+ exports org.apache.arrow.adapter.avro.producers.logical;
exports org.apache.arrow.adapter.avro;
requires org.apache.arrow.memory.core;
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java
new file mode 100644
index 000000000..6f0cb5cff
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/ArrowToAvroUtils.java
@@ -0,0 +1,524 @@
+/*
+ * 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.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.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.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.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.LogicalType;
+import org.apache.avro.LogicalTypes;
+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.
+ *
+ *
This method currently performs following type mapping for Avro data types to corresponding
+ * Arrow data types.
+ *
+ *
+ * Arrow type | Avro encoding |
+ *
+ * ArrowType.Null | NULL |
+ * ArrowType.Bool | BOOLEAN |
+ * ArrowType.Int(64 bit, unsigned 32 bit) | LONG |
+ * ArrowType.Int(signed 32 bit, < 32 bit) | INT |
+ * ArrowType.FloatingPoint(double) | DOUBLE |
+ * ArrowType.FloatingPoint(single, half) | FLOAT |
+ * ArrowType.Utf8 | STRING |
+ * ArrowType.LargeUtf8 | STRING |
+ * ArrowType.Binary | BYTES |
+ * ArrowType.LargeBinary | BYTES |
+ * ArrowType.FixedSizeBinary | FIXED |
+ * ArrowType.Decimal | decimal (FIXED) |
+ * ArrowType.Date | date (INT) |
+ * ArrowType.Time (SEC | MILLI) | time-millis (INT) |
+ * ArrowType.Time (MICRO | NANO) | time-micros (LONG) |
+ * ArrowType.Timestamp (NANOSECONDS, TZ != NULL) | time-nanos (LONG) |
+ * ArrowType.Timestamp (MICROSECONDS, TZ != NULL) | time-micros (LONG) |
+ * ArrowType.Timestamp (MILLISECONDS | SECONDS, TZ != NULL) | time-millis (LONG) |
+ * ArrowType.Timestamp (NANOSECONDS, TZ == NULL) | local-time-nanos (LONG) |
+ * ArrowType.Timestamp (MICROSECONDS, TZ == NULL) | local-time-micros (LONG) |
+ * ArrowType.Timestamp (MILLISECONDS | SECONDS, TZ == NULL) | local-time-millis (LONG) |
+ * ArrowType.Duration | duration (FIXED) |
+ * ArrowType.Interval | duration (FIXED) |
+ * ArrowType.Struct | record |
+ * ArrowType.List | array |
+ * ArrowType.LargeList | array |
+ * ArrowType.FixedSizeList | array |
+ * ArrowType.Map | map |
+ * ArrowType.Union | union |
+ *
+ *
+ *
+ * Nullable fields are represented as a union of [base-type | null]. Special treatment is given
+ * to nullability of unions - a union is considered nullable if any of its child fields are
+ * nullable. The schema for a nullable union will always contain a null type as its first member,
+ * with none of the child types being nullable.
+ *
+ *
List fields must contain precisely one child field, which may be nullable. Map fields are
+ * represented as a list of structs, where the struct fields are "key" and "value". The key field
+ * must always be of type STRING (Utf8) and cannot be nullable. The value can be of any type and
+ * may be nullable. Record types must contain at least one child field and cannot contain multiple
+ * fields with the same name
+ *
+ * @param arrowFields The arrow fields used to generate the Avro schema
+ * @param typeName Name of the top level Avro record type
+ * @param namespace Namespace of the top level Avro record type
+ * @return An Avro record schema for the given list of fields, with the specified name and
+ * namespace
+ */
+ public static Schema createAvroSchema(
+ List arrowFields, String typeName, String namespace) {
+ SchemaBuilder.RecordBuilder assembler =
+ SchemaBuilder.record(typeName).namespace(namespace);
+ return buildRecordSchema(assembler, arrowFields, namespace);
+ }
+
+ /** Overload provided for convenience, sets namespace = null. */
+ public static Schema createAvroSchema(List arrowFields, String typeName) {
+ return createAvroSchema(arrowFields, typeName, null);
+ }
+
+ /** Overload provided for convenience, sets name = GENERIC_RECORD_TYPE_NAME. */
+ public static Schema createAvroSchema(List arrowFields) {
+ return createAvroSchema(arrowFields, GENERIC_RECORD_TYPE_NAME);
+ }
+
+ private static T buildRecordSchema(
+ SchemaBuilder.RecordBuilder builder, List fields, String namespace) {
+ if (fields.isEmpty()) {
+ throw new IllegalArgumentException("Record field must have at least one child field");
+ }
+ SchemaBuilder.FieldAssembler assembler = builder.namespace(namespace).fields();
+ for (Field field : fields) {
+ assembler = buildFieldSchema(assembler, field, namespace);
+ }
+ return assembler.endRecord();
+ }
+
+ private static SchemaBuilder.FieldAssembler buildFieldSchema(
+ SchemaBuilder.FieldAssembler assembler, Field field, String namespace) {
+
+ return assembler
+ .name(field.getName())
+ .type(buildTypeSchema(SchemaBuilder.builder(), field, namespace))
+ .noDefault();
+ }
+
+ private static T buildTypeSchema(
+ SchemaBuilder.TypeBuilder builder, Field field, String namespace) {
+
+ // Nullable unions need special handling, since union types cannot be directly nested
+ if (field.getType().getTypeID() == ArrowType.ArrowTypeID.Union) {
+ boolean unionNullable = field.getChildren().stream().anyMatch(Field::isNullable);
+ if (unionNullable) {
+ SchemaBuilder.UnionAccumulator union = builder.unionOf().nullType();
+ return addTypesToUnion(union, field.getChildren(), namespace);
+ } else {
+ Field headType = field.getChildren().get(0);
+ List tailTypes = field.getChildren().subList(1, field.getChildren().size());
+ SchemaBuilder.UnionAccumulator union =
+ buildBaseTypeSchema(builder.unionOf(), headType, namespace);
+ return addTypesToUnion(union, tailTypes, namespace);
+ }
+ } else if (field.isNullable()) {
+ return buildBaseTypeSchema(builder.nullable(), field, namespace);
+ } else {
+ return buildBaseTypeSchema(builder, field, namespace);
+ }
+ }
+
+ private static T buildArraySchema(
+ SchemaBuilder.ArrayBuilder builder, Field listField, String namespace) {
+ if (listField.getChildren().size() != 1) {
+ throw new IllegalArgumentException("List field must have exactly one child field");
+ }
+ Field itemField = listField.getChildren().get(0);
+ return buildTypeSchema(builder.items(), itemField, namespace);
+ }
+
+ private static T buildMapSchema(
+ SchemaBuilder.MapBuilder builder, Field mapField, String namespace) {
+ if (mapField.getChildren().size() != 1) {
+ throw new IllegalArgumentException("Map field must have exactly one child field");
+ }
+ Field entriesField = mapField.getChildren().get(0);
+ if (mapField.getChildren().size() != 1) {
+ throw new IllegalArgumentException("Map entries must have exactly two child fields");
+ }
+ Field keyField = entriesField.getChildren().get(0);
+ Field valueField = entriesField.getChildren().get(1);
+ if (keyField.getType().getTypeID() != ArrowType.ArrowTypeID.Utf8 || keyField.isNullable()) {
+ throw new IllegalArgumentException(
+ "Map keys must be of type string and cannot be nullable for conversion to Avro");
+ }
+ return buildTypeSchema(builder.values(), valueField, namespace);
+ }
+
+ private static T buildBaseTypeSchema(
+ SchemaBuilder.BaseTypeBuilder builder, Field field, String namespace) {
+
+ ArrowType.ArrowTypeID typeID = field.getType().getTypeID();
+
+ switch (typeID) {
+ case Null:
+ return builder.nullType();
+
+ case Bool:
+ return builder.booleanType();
+
+ case Int:
+ ArrowType.Int intType = (ArrowType.Int) field.getType();
+ if (intType.getBitWidth() > 32 || (intType.getBitWidth() == 32 && !intType.getIsSigned())) {
+ return builder.longType();
+ } else {
+ return builder.intType();
+ }
+
+ case FloatingPoint:
+ ArrowType.FloatingPoint floatType = (ArrowType.FloatingPoint) field.getType();
+ if (floatType.getPrecision() == FloatingPointPrecision.DOUBLE) {
+ return builder.doubleType();
+ } else {
+ return builder.floatType();
+ }
+
+ case Utf8:
+ return builder.stringType();
+
+ case Binary:
+ return builder.bytesType();
+
+ case FixedSizeBinary:
+ ArrowType.FixedSizeBinary fixedType = (ArrowType.FixedSizeBinary) field.getType();
+ String fixedTypeName = field.getName();
+ int fixedTypeWidth = fixedType.getByteWidth();
+ return builder.fixed(fixedTypeName).size(fixedTypeWidth);
+
+ case Decimal:
+ ArrowType.Decimal decimalType = (ArrowType.Decimal) field.getType();
+ return builder.type(
+ LogicalTypes.decimal(decimalType.getPrecision(), decimalType.getScale())
+ .addToSchema(
+ Schema.createFixed(
+ field.getName(), namespace, "", decimalType.getBitWidth() / 8)));
+
+ case Date:
+ return builder.type(LogicalTypes.date().addToSchema(Schema.create(Schema.Type.INT)));
+
+ case Time:
+ ArrowType.Time timeType = (ArrowType.Time) field.getType();
+ if ((timeType.getUnit() == TimeUnit.SECOND || timeType.getUnit() == TimeUnit.MILLISECOND)) {
+ // Second and millisecond time types are encoded as time-millis (INT)
+ return builder.type(
+ LogicalTypes.timeMillis().addToSchema(Schema.create(Schema.Type.INT)));
+ } else {
+ // All other time types (micro, nano) are encoded as time-micros (LONG)
+ return builder.type(
+ LogicalTypes.timeMicros().addToSchema(Schema.create(Schema.Type.LONG)));
+ }
+
+ case Timestamp:
+ ArrowType.Timestamp timestampType = (ArrowType.Timestamp) field.getType();
+ LogicalType timestampLogicalType = timestampLogicalType(timestampType);
+ return builder.type(timestampLogicalType.addToSchema(Schema.create(Schema.Type.LONG)));
+
+ case Struct:
+ String childNamespace =
+ namespace == null ? field.getName() : namespace + "." + field.getName();
+ return buildRecordSchema(
+ builder.record(field.getName()), field.getChildren(), childNamespace);
+
+ case List:
+ case FixedSizeList:
+ return buildArraySchema(builder.array(), field, namespace);
+
+ case Map:
+ return buildMapSchema(builder.map(), field, namespace);
+
+ default:
+ throw new IllegalArgumentException(
+ "Element type not supported for Avro conversion: " + typeID.name());
+ }
+ }
+
+ private static T addTypesToUnion(
+ SchemaBuilder.UnionAccumulator accumulator, List unionFields, String namespace) {
+ for (var field : unionFields) {
+ accumulator = buildBaseTypeSchema(accumulator.and(), field, namespace);
+ }
+ return accumulator.endUnion();
+ }
+
+ private static LogicalType timestampLogicalType(ArrowType.Timestamp timestampType) {
+ boolean zoneAware = timestampType.getTimezone() != null;
+ if (timestampType.getUnit() == TimeUnit.NANOSECOND) {
+ return zoneAware ? LogicalTypes.timestampNanos() : LogicalTypes.localTimestampNanos();
+ } else if (timestampType.getUnit() == TimeUnit.MICROSECOND) {
+ return zoneAware ? LogicalTypes.timestampMicros() : LogicalTypes.localTimestampMicros();
+ } else {
+ // Timestamp in seconds will be cast to milliseconds, Avro does not support seconds
+ return zoneAware ? LogicalTypes.timestampMillis() : LogicalTypes.localTimestampMillis();
+ }
+ }
+
+ /**
+ * Create a composite Avro producer for a set of field vectors (typically the root set of a VSR).
+ *
+ * @param vectors The vectors that will be used to produce Avro data
+ * @return The resulting composite Avro producer
+ */
+ public static CompositeAvroProducer createCompositeProducer(List vectors) {
+
+ List> producers = new ArrayList<>(vectors.size());
+
+ for (FieldVector vector : vectors) {
+ BaseAvroProducer extends FieldVector> producer = createProducer(vector);
+ producers.add(producer);
+ }
+
+ return new CompositeAvroProducer(producers);
+ }
+
+ private static BaseAvroProducer> createProducer(FieldVector vector) {
+ boolean nullable = vector.getField().isNullable();
+ return createProducer(vector, nullable);
+ }
+
+ private static BaseAvroProducer> createProducer(FieldVector vector, boolean nullable) {
+
+ Preconditions.checkNotNull(vector, "Arrow vector object can't be null");
+
+ final Types.MinorType minorType = vector.getMinorType();
+
+ // Avro understands nullable types as a union of type | null
+ // Most nullable fields in a VSR will not be unions, so provide a special wrapper
+ if (nullable && minorType != Types.MinorType.UNION) {
+ final BaseAvroProducer> innerProducer = createProducer(vector, false);
+ return new AvroNullableProducer<>(innerProducer);
+ }
+
+ switch (minorType) {
+ case NULL:
+ return new AvroNullProducer((NullVector) vector);
+ case BIT:
+ return new AvroBooleanProducer((BitVector) vector);
+ case TINYINT:
+ return new AvroTinyIntProducer((TinyIntVector) vector);
+ case SMALLINT:
+ return new AvroSmallIntProducer((SmallIntVector) vector);
+ case INT:
+ return new AvroIntProducer((IntVector) vector);
+ case BIGINT:
+ return new AvroBigIntProducer((BigIntVector) vector);
+ case UINT1:
+ return new AvroUint1Producer((UInt1Vector) vector);
+ case UINT2:
+ return new AvroUint2Producer((UInt2Vector) vector);
+ case UINT4:
+ return new AvroUint4Producer((UInt4Vector) vector);
+ case UINT8:
+ return new AvroUint8Producer((UInt8Vector) vector);
+ case FLOAT2:
+ return new AvroFloat2Producer((Float2Vector) vector);
+ case FLOAT4:
+ return new AvroFloat4Producer((Float4Vector) vector);
+ case FLOAT8:
+ return new AvroFloat8Producer((Float8Vector) vector);
+ case VARBINARY:
+ return new AvroBytesProducer((VarBinaryVector) vector);
+ case FIXEDSIZEBINARY:
+ return new AvroFixedSizeBinaryProducer((FixedSizeBinaryVector) vector);
+ case VARCHAR:
+ return new AvroStringProducer((VarCharVector) vector);
+
+ // Logical types
+
+ case DECIMAL:
+ return new AvroDecimalProducer((DecimalVector) vector);
+ case DECIMAL256:
+ return new AvroDecimal256Producer((Decimal256Vector) vector);
+ case DATEDAY:
+ return new AvroDateDayProducer((DateDayVector) vector);
+ case DATEMILLI:
+ return new AvroDateMilliProducer((DateMilliVector) vector);
+ case TIMESEC:
+ return new AvroTimeSecProducer((TimeSecVector) vector);
+ case TIMEMILLI:
+ return new AvroTimeMilliProducer((TimeMilliVector) vector);
+ case TIMEMICRO:
+ return new AvroTimeMicroProducer((TimeMicroVector) vector);
+ case TIMENANO:
+ return new AvroTimeNanoProducer((TimeNanoVector) vector);
+ case TIMESTAMPSEC:
+ return new AvroTimestampSecProducer((TimeStampSecVector) vector);
+ case TIMESTAMPMILLI:
+ return new AvroTimestampMilliProducer((TimeStampMilliVector) vector);
+ case TIMESTAMPMICRO:
+ return new AvroTimestampMicroProducer((TimeStampMicroVector) vector);
+ case TIMESTAMPNANO:
+ return new AvroTimestampNanoProducer((TimeStampNanoVector) vector);
+ case TIMESTAMPSECTZ:
+ return new AvroTimestampSecTzProducer((TimeStampSecTZVector) vector);
+ case TIMESTAMPMILLITZ:
+ return new AvroTimestampMilliTzProducer((TimeStampMilliTZVector) vector);
+ case TIMESTAMPMICROTZ:
+ return new AvroTimestampMicroTzProducer((TimeStampMicroTZVector) vector);
+ case TIMESTAMPNANOTZ:
+ return new AvroTimestampNanoTzProducer((TimeStampNanoTZVector) vector);
+
+ // Complex types
+
+ case STRUCT:
+ StructVector structVector = (StructVector) vector;
+ List childVectors = structVector.getChildrenFromFields();
+ Producer>[] childProducers = new Producer>[childVectors.size()];
+ for (int i = 0; i < childVectors.size(); i++) {
+ FieldVector childVector = childVectors.get(i);
+ childProducers[i] = createProducer(childVector, childVector.getField().isNullable());
+ }
+ return new AvroStructProducer(structVector, childProducers);
+
+ case LIST:
+ ListVector listVector = (ListVector) vector;
+ FieldVector itemVector = listVector.getDataVector();
+ Producer> itemProducer = createProducer(itemVector, itemVector.getField().isNullable());
+ return new AvroListProducer(listVector, itemProducer);
+
+ case FIXED_SIZE_LIST:
+ FixedSizeListVector fixedListVector = (FixedSizeListVector) vector;
+ FieldVector fixedItemVector = fixedListVector.getDataVector();
+ Producer> fixedItemProducer =
+ createProducer(fixedItemVector, fixedItemVector.getField().isNullable());
+ return new AvroFixedSizeListProducer(fixedListVector, fixedItemProducer);
+
+ case MAP:
+ MapVector mapVector = (MapVector) vector;
+ StructVector entryVector = (StructVector) mapVector.getDataVector();
+ Types.MinorType keyType = entryVector.getChildrenFromFields().get(0).getMinorType();
+ if (keyType != Types.MinorType.VARCHAR) {
+ throw new IllegalArgumentException("MAP key type must be VARCHAR for Avro encoding");
+ }
+ VarCharVector keyVector = (VarCharVector) entryVector.getChildrenFromFields().get(0);
+ FieldVector valueVector = entryVector.getChildrenFromFields().get(1);
+ Producer> keyProducer = new AvroStringProducer(keyVector);
+ Producer> valueProducer =
+ createProducer(valueVector, valueVector.getField().isNullable());
+ Producer> entryProducer =
+ new AvroStructProducer(entryVector, new Producer>[] {keyProducer, valueProducer});
+ return new AvroMapProducer(mapVector, entryProducer);
+
+ // Support for UNION and DENSEUNION is not currently available
+ // This is pending fixes in the implementation of the union vectors themselves
+ // https://github.com/apache/arrow-java/issues/108
+
+ default:
+ // Not all Arrow types are supported for encoding (yet)!
+ String error =
+ String.format(
+ "Encoding Arrow type %s to Avro is not currently supported", minorType.name());
+ throw new UnsupportedOperationException(error);
+ }
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBigIntProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBigIntProducer.java
new file mode 100644
index 000000000..9712e157c
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBigIntProducer.java
@@ -0,0 +1,52 @@
+/*
+ * 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.BaseFixedWidthVector;
+import org.apache.arrow.vector.BigIntVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces long values from a {@link BigIntVector}, writes data to an Avro encoder.
+ *
+ * Logical types are also supported, for vectors derived from {@link BaseFixedWidthVector} where
+ * the internal representation matches BigIntVector and requires no conversion.
+ */
+public class AvroBigIntProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroBigIntProducer. */
+ public AvroBigIntProducer(BigIntVector vector) {
+ super(vector);
+ }
+
+ /** Protected constructor for logical types with a long representation. */
+ protected AvroBigIntProducer(BaseFixedWidthVector vector) {
+ super(vector);
+ if (vector.getTypeWidth() != BigIntVector.TYPE_WIDTH) {
+ throw new IllegalArgumentException(
+ "AvroBigIntProducer requires type width = " + BigIntVector.TYPE_WIDTH);
+ }
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long value = vector.getDataBuffer().getLong(currentIndex * (long) BigIntVector.TYPE_WIDTH);
+ encoder.writeLong(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBooleanProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBooleanProducer.java
new file mode 100644
index 000000000..523ddf110
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBooleanProducer.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.BitVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces boolean values from a {@link BitVector}, writes data to an Avro encoder.
+ */
+public class AvroBooleanProducer extends BaseAvroProducer {
+
+ /** Instantiate am AvroBooleanProducer. */
+ public AvroBooleanProducer(BitVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ int bitValue = vector.get(currentIndex++);
+ encoder.writeBoolean(bitValue != 0);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBytesProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBytesProducer.java
new file mode 100644
index 000000000..e1fe6dddc
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroBytesProducer.java
@@ -0,0 +1,45 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import org.apache.arrow.vector.VarBinaryVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces byte array values from a {@link VarBinaryVector}, writes data to an Avro
+ * encoder.
+ */
+public class AvroBytesProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroBytesProducer. */
+ public AvroBytesProducer(VarBinaryVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ // The nio ByteBuffer is created once per call, but underlying data is not copied
+ long offset = vector.getStartOffset(currentIndex);
+ long endOffset = vector.getEndOffset(currentIndex);
+ int length = (int) (endOffset - offset);
+ ByteBuffer nioBuffer = vector.getDataBuffer().nioBuffer(offset, length);
+ encoder.writeBytes(nioBuffer);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroEnumProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroEnumProducer.java
new file mode 100644
index 000000000..068566493
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroEnumProducer.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.IntVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces enum values from a dictionary-encoded {@link IntVector}, writes data to an
+ * Avro encoder.
+ */
+public class AvroEnumProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroEnumProducer. */
+ public AvroEnumProducer(IntVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ encoder.writeEnum(vector.get(currentIndex++));
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeBinaryProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeBinaryProducer.java
new file mode 100644
index 000000000..9fb877cfa
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeBinaryProducer.java
@@ -0,0 +1,54 @@
+/*
+ * 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.BaseFixedWidthVector;
+import org.apache.arrow.vector.FixedSizeBinaryVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces fixed-size binary values from a {@link FixedSizeBinaryVector}, writes data
+ * to an Avro encoder.
+ *
+ * Logical types are also supported, for vectors derived from {@link BaseFixedWidthVector} where
+ * the internal representation is fixed width bytes and requires no conversion.
+ */
+public class AvroFixedSizeBinaryProducer extends BaseAvroProducer {
+
+ private final byte[] reuseBytes;
+
+ /** Instantiate an AvroFixedSizeBinaryProducer. */
+ public AvroFixedSizeBinaryProducer(FixedSizeBinaryVector vector) {
+ super(vector);
+ reuseBytes = new byte[vector.getTypeWidth()];
+ }
+
+ /** Protected constructor for logical types with a fixed width representation. */
+ protected AvroFixedSizeBinaryProducer(BaseFixedWidthVector vector) {
+ super(vector);
+ reuseBytes = new byte[vector.getTypeWidth()];
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long offset = (long) currentIndex * vector.getTypeWidth();
+ vector.getDataBuffer().getBytes(offset, reuseBytes);
+ encoder.writeFixed(reuseBytes);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeListProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeListProducer.java
new file mode 100644
index 000000000..acb6fb8c0
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFixedSizeListProducer.java
@@ -0,0 +1,78 @@
+/*
+ * 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.FixedSizeListVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces array values from a {@link FixedSizeListVector}, writes data to an avro
+ * encoder.
+ */
+public class AvroFixedSizeListProducer extends BaseAvroProducer {
+
+ private final Producer extends FieldVector> delegate;
+
+ /** Instantiate an AvroFixedSizeListProducer. */
+ public AvroFixedSizeListProducer(
+ FixedSizeListVector vector, Producer extends FieldVector> delegate) {
+ super(vector);
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+
+ encoder.writeArrayStart();
+ encoder.setItemCount(vector.getListSize());
+
+ for (int i = 0; i < vector.getListSize(); i++) {
+ encoder.startItem();
+ delegate.produce(encoder);
+ }
+
+ encoder.writeArrayEnd();
+ currentIndex++;
+ }
+
+ @Override
+ public void skipNull() {
+ super.skipNull();
+ // Child vector contains a fixed number of elements for each entry
+ int childIndex = currentIndex * vector.getListSize();
+ delegate.setPosition(childIndex);
+ }
+
+ @Override
+ public void setPosition(int index) {
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds");
+ }
+ super.setPosition(index);
+ // Child vector contains a fixed number of elements for each entry
+ int childIndex = currentIndex * vector.getListSize();
+ delegate.setPosition(childIndex);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void resetValueVector(FixedSizeListVector vector) {
+ ((Producer) delegate).resetValueVector(vector.getDataVector());
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat2Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat2Producer.java
new file mode 100644
index 000000000..07e5ea359
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat2Producer.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.memory.util.Float16;
+import org.apache.arrow.vector.Float2Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces float values from a {@link Float2Vector}, writes data to an Avro encoder.
+ */
+public class AvroFloat2Producer extends BaseAvroProducer {
+
+ /** Instantiate an AvroFloat2Producer. */
+ public AvroFloat2Producer(Float2Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ short rawValue = vector.getDataBuffer().getShort(currentIndex * (long) Float2Vector.TYPE_WIDTH);
+ encoder.writeFloat(Float16.toFloat(rawValue));
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat4Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat4Producer.java
new file mode 100644
index 000000000..5121ba3a1
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat4Producer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.Float4Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces float values from a {@link Float4Vector}, writes data to an Avro encoder.
+ */
+public class AvroFloat4Producer extends BaseAvroProducer {
+
+ /** Instantiate an AvroFloat4Producer. */
+ public AvroFloat4Producer(Float4Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ float value = vector.getDataBuffer().getFloat(currentIndex * (long) Float4Vector.TYPE_WIDTH);
+ encoder.writeFloat(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat8Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat8Producer.java
new file mode 100644
index 000000000..05fca750b
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroFloat8Producer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.Float8Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces double values from a {@link Float8Vector}, writes data to an Avro encoder.
+ */
+public class AvroFloat8Producer extends BaseAvroProducer {
+
+ /** Instantiate an AvroFloat8Producer. */
+ public AvroFloat8Producer(Float8Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ double value = vector.getDataBuffer().getDouble(currentIndex * (long) Float8Vector.TYPE_WIDTH);
+ encoder.writeDouble(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroIntProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroIntProducer.java
new file mode 100644
index 000000000..4c9cc9b71
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroIntProducer.java
@@ -0,0 +1,52 @@
+/*
+ * 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.BaseFixedWidthVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces int values from an {@link IntVector}, writes data to an avro encoder.
+ *
+ * Logical types are also supported, for vectors derived from {@link BaseFixedWidthVector} where
+ * the internal representation matches IntVector and requires no conversion.
+ */
+public class AvroIntProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroIntConsumer. */
+ public AvroIntProducer(IntVector vector) {
+ super(vector);
+ }
+
+ /** Protected constructor for a logical types with an integer representation. */
+ protected AvroIntProducer(BaseFixedWidthVector vector) {
+ super(vector);
+ if (vector.getTypeWidth() != IntVector.TYPE_WIDTH) {
+ throw new IllegalArgumentException(
+ "AvroIntProducer requires type width = " + IntVector.TYPE_WIDTH);
+ }
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ int value = vector.getDataBuffer().getInt(currentIndex * (long) IntVector.TYPE_WIDTH);
+ encoder.writeInt(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroListProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroListProducer.java
new file mode 100644
index 000000000..10cfe9549
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroListProducer.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.ListVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces array values from a {@link ListVector}, writes data to an avro encoder.
+ */
+public class AvroListProducer extends BaseAvroProducer {
+
+ private final Producer extends FieldVector> delegate;
+
+ /** Instantiate an AvroListProducer. */
+ public AvroListProducer(ListVector vector, Producer extends FieldVector> delegate) {
+ super(vector);
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+
+ int startOffset = vector.getOffsetBuffer().getInt(currentIndex * (long) Integer.BYTES);
+ int endOffset = vector.getOffsetBuffer().getInt((currentIndex + 1) * (long) Integer.BYTES);
+ int nItems = endOffset - startOffset;
+
+ encoder.writeArrayStart();
+ encoder.setItemCount(nItems);
+
+ for (int i = 0; i < nItems; i++) {
+ encoder.startItem();
+ delegate.produce(encoder);
+ }
+
+ encoder.writeArrayEnd();
+ currentIndex++;
+ }
+
+ // Do not override skipNull(), delegate will not have an entry if the list is null
+
+ @Override
+ public void setPosition(int index) {
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds");
+ }
+ int delegateOffset = vector.getOffsetBuffer().getInt(index * (long) Integer.BYTES);
+ delegate.setPosition(delegateOffset);
+ super.setPosition(index);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void resetValueVector(ListVector vector) {
+ ((Producer) delegate).resetValueVector(vector.getDataVector());
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroMapProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroMapProducer.java
new file mode 100644
index 000000000..568d5b62e
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroMapProducer.java
@@ -0,0 +1,71 @@
+/*
+ * 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.MapVector;
+import org.apache.avro.io.Encoder;
+
+/** Producer which produces map type values to avro encoder. Write the data to {@link MapVector}. */
+public class AvroMapProducer extends BaseAvroProducer {
+
+ private final Producer extends FieldVector> delegate;
+
+ /** Instantiate a AvroMapProducer. */
+ public AvroMapProducer(MapVector vector, Producer extends FieldVector> delegate) {
+ super(vector);
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+
+ int startOffset = vector.getOffsetBuffer().getInt(currentIndex * (long) Integer.BYTES);
+ int endOffset = vector.getOffsetBuffer().getInt((currentIndex + 1) * (long) Integer.BYTES);
+ int nEntries = endOffset - startOffset;
+
+ encoder.writeMapStart();
+ encoder.setItemCount(nEntries);
+
+ for (int i = 0; i < nEntries; i++) {
+ encoder.startItem();
+ delegate.produce(encoder);
+ }
+
+ encoder.writeMapEnd();
+ currentIndex++;
+ }
+
+ // Do not override skipNull(), delegate will not have an entry if the map is null
+
+ @Override
+ public void setPosition(int index) {
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds");
+ }
+ int delegateOffset = vector.getOffsetBuffer().getInt(index * (long) Integer.BYTES);
+ delegate.setPosition(delegateOffset);
+ super.setPosition(index);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void resetValueVector(MapVector vector) {
+ ((Producer) delegate).resetValueVector(vector.getDataVector());
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullProducer.java
new file mode 100644
index 000000000..1bd1e891f
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullProducer.java
@@ -0,0 +1,36 @@
+/*
+ * 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.NullVector;
+import org.apache.avro.io.Encoder;
+
+/** Producer that produces null values from a {@link NullVector}, writes data to an Avro encoder. */
+public class AvroNullProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroNullProducer. */
+ public AvroNullProducer(NullVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ encoder.writeNull();
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullableProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullableProducer.java
new file mode 100644
index 000000000..f4215dbf8
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroNullableProducer.java
@@ -0,0 +1,77 @@
+/*
+ * 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;
+
+/**
+ * Producer wrapper which producers nullable types to an avro encoder. Write the data to the
+ * underlying {@link FieldVector}.
+ *
+ * @param The vector within producer or its delegate, used for partially produce purpose.
+ */
+public class AvroNullableProducer extends BaseAvroProducer {
+
+ private final Producer delegate;
+
+ /** Instantiate a AvroNullableProducer. */
+ public AvroNullableProducer(Producer delegate) {
+ super(delegate.getVector());
+ this.delegate = delegate;
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ if (vector.isNull(currentIndex)) {
+ encoder.writeInt(1);
+ encoder.writeNull();
+ delegate.skipNull();
+ } else {
+ encoder.writeInt(0);
+ delegate.produce(encoder);
+ }
+ currentIndex++;
+ }
+
+ @Override
+ public void skipNull() {
+ // Can be called by containers of nullable types
+ delegate.skipNull();
+ currentIndex++;
+ }
+
+ @Override
+ public void setPosition(int index) {
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds");
+ }
+ delegate.setPosition(index);
+ super.setPosition(index);
+ }
+
+ @Override
+ public void resetValueVector(T vector) {
+ delegate.resetValueVector(vector);
+ }
+
+ @Override
+ public T getVector() {
+ return delegate.getVector();
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroSmallIntProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroSmallIntProducer.java
new file mode 100644
index 000000000..9c37750d9
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroSmallIntProducer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.SmallIntVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces int values from an {@link SmallIntVector}, writes data to an avro encoder.
+ */
+public class AvroSmallIntProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroSmallIntProducer. */
+ public AvroSmallIntProducer(SmallIntVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ short value = vector.getDataBuffer().getShort(currentIndex * (long) SmallIntVector.TYPE_WIDTH);
+ encoder.writeInt(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStringProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStringProducer.java
new file mode 100644
index 000000000..19e165cd1
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStringProducer.java
@@ -0,0 +1,48 @@
+/*
+ * 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 java.nio.ByteBuffer;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces string values from a {@link VarCharVector}, writes data to an Avro
+ * encoder.
+ */
+public class AvroStringProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroStringProducer. */
+ public AvroStringProducer(VarCharVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+
+ int start = vector.getStartOffset(currentIndex);
+ int end = vector.getEndOffset(currentIndex);
+ int length = end - start;
+
+ // The nio ByteBuffer is created once per call, but underlying data is not copied
+ ByteBuffer nioBuffer = vector.getDataBuffer().nioBuffer(start, length);
+ encoder.writeBytes(nioBuffer);
+
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStructProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStructProducer.java
new file mode 100644
index 000000000..86c1949bf
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroStructProducer.java
@@ -0,0 +1,75 @@
+/*
+ * 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 {
+
+ 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) {
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds: " + index);
+ }
+ for (Producer> delegate : delegates) {
+ delegate.setPosition(index);
+ }
+ super.setPosition(index);
+ }
+
+ @Override
+ @SuppressWarnings("unchecked")
+ public void resetValueVector(StructVector vector) {
+ for (int i = 0; i < delegates.length; i++) {
+ Producer delegate = (Producer) delegates[i];
+ delegate.resetValueVector(vector.getChildrenFromFields().get(i));
+ }
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroTinyIntProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroTinyIntProducer.java
new file mode 100644
index 000000000..30a80e509
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroTinyIntProducer.java
@@ -0,0 +1,39 @@
+/*
+ * 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.TinyIntVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces int values from an {@link TinyIntVector}, writes data to an avro encoder.
+ */
+public class AvroTinyIntProducer extends BaseAvroProducer {
+
+ /** Instantiate an AvroTinyIntProducer. */
+ public AvroTinyIntProducer(TinyIntVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ byte value = vector.getDataBuffer().getByte(currentIndex * (long) TinyIntVector.TYPE_WIDTH);
+ encoder.writeInt(value);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint1Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint1Producer.java
new file mode 100644
index 000000000..83cbc9ef8
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint1Producer.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.UInt1Vector;
+import org.apache.avro.io.Encoder;
+
+/** Producer that produces int values from a {@link UInt1Vector}, writes data to an avro encoder. */
+public class AvroUint1Producer extends BaseAvroProducer {
+
+ /** Instantiate an AvroUint1Producer. */
+ public AvroUint1Producer(UInt1Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ byte unsigned = vector.getDataBuffer().getByte(currentIndex * (long) UInt1Vector.TYPE_WIDTH);
+ int unsignedInt = Byte.toUnsignedInt(unsigned);
+ encoder.writeInt(unsignedInt);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint2Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint2Producer.java
new file mode 100644
index 000000000..1e30c82cd
--- /dev/null
+++ b/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 {
+
+ /** 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 unsignedInt = Short.toUnsignedInt(unsigned);
+ encoder.writeInt(unsignedInt);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint4Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint4Producer.java
new file mode 100644
index 000000000..63f78429d
--- /dev/null
+++ b/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 {
+
+ /** 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 unsignedLong = Integer.toUnsignedLong(unsigned);
+ encoder.writeLong(unsignedLong);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint8Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint8Producer.java
new file mode 100644
index 000000000..819b4d414
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/AvroUint8Producer.java
@@ -0,0 +1,42 @@
+/*
+ * 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.UInt8Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces long values from a {@link UInt8Vector}, writes data to an avro encoder.
+ */
+public class AvroUint8Producer extends BaseAvroProducer {
+
+ /** Instantiate an AvroUint8Producer. */
+ public AvroUint8Producer(UInt8Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long unsigned = vector.getDataBuffer().getLong(currentIndex * (long) UInt8Vector.TYPE_WIDTH);
+ if (unsigned < 0) {
+ throw new ArithmeticException("Unsigned long value is too large for Avro encoding");
+ }
+ encoder.writeLong(unsigned);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/BaseAvroProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/BaseAvroProducer.java
new file mode 100644
index 000000000..30c004bdc
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/BaseAvroProducer.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 org.apache.arrow.vector.FieldVector;
+
+/**
+ * Base class for avro producers.
+ *
+ * @param vector type.
+ */
+public abstract class BaseAvroProducer implements Producer {
+
+ protected T vector;
+ protected int currentIndex;
+
+ /**
+ * Constructs a base avro consumer.
+ *
+ * @param vector the vector to consume.
+ */
+ protected BaseAvroProducer(T vector) {
+ this.vector = vector;
+ }
+
+ @Override
+ public void skipNull() {
+ currentIndex++;
+ }
+
+ /**
+ * Sets the current index for this producer against the underlying vector.
+ *
+ * For a vector of length N, the valid range is [0, N] inclusive. Setting index = N signifies
+ * that no further data is available for production (this is the state the produce will be in when
+ * production for the current vector is complete).
+ *
+ * @param index New current index for the producer
+ */
+ @Override
+ public void setPosition(int index) {
+ // currentIndex == value count is a valid state, no more values will be produced
+ if (index < 0 || index > vector.getValueCount()) {
+ throw new IllegalArgumentException("Index out of bounds");
+ }
+ currentIndex = index;
+ }
+
+ @Override
+ public void resetValueVector(T vector) {
+ this.vector = vector;
+ this.currentIndex = 0;
+ }
+
+ @Override
+ public T getVector() {
+ return vector;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/CompositeAvroProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/CompositeAvroProducer.java
new file mode 100644
index 000000000..d1ed50610
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/CompositeAvroProducer.java
@@ -0,0 +1,54 @@
+/*
+ * 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 java.util.List;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.avro.io.Encoder;
+
+/** Composite producer which holds all producers. It manages the produce and cleanup process. */
+public class CompositeAvroProducer {
+
+ private final List> producers;
+
+ public CompositeAvroProducer(List> producers) {
+ this.producers = producers;
+ }
+
+ public List> getProducers() {
+ return producers;
+ }
+
+ /** Produce encoder data. */
+ public void produce(Encoder encoder) throws IOException {
+ for (Producer extends FieldVector> producer : producers) {
+ producer.produce(encoder);
+ }
+ }
+
+ /** Reset vector of consumers with the given {@link VectorSchemaRoot}. */
+ @SuppressWarnings({"unchecked", "rawtypes"})
+ public void resetProducerVectors(VectorSchemaRoot root) {
+ // This method assumes that the VSR matches the constructed set of producers
+ int index = 0;
+ for (Producer producer : producers) {
+ producer.resetValueVector(root.getFieldVectors().get(index));
+ }
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/Producer.java
new file mode 100644
index 000000000..aed254334
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/Producer.java
@@ -0,0 +1,49 @@
+/*
+ * 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 The vector within producer or its delegate, used for partially produce purpose.
+ */
+public interface Producer {
+
+ /**
+ * 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. */
+ void resetValueVector(T vector);
+
+ /** Get the vector within the producer. */
+ T getVector();
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateDayProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateDayProducer.java
new file mode 100644
index 000000000..36680fb19
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateDayProducer.java
@@ -0,0 +1,33 @@
+/*
+ * 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.DateDayVector;
+
+/**
+ * Producer that produces date values from a {@link DateDayVector}, writes data to an Avro encoder.
+ */
+public class AvroDateDayProducer extends AvroIntProducer {
+
+ // Date stored as integer number of days, matches Avro date type
+
+ /** Instantiate an AvroDateProducer. */
+ public AvroDateDayProducer(DateDayVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateMilliProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateMilliProducer.java
new file mode 100644
index 000000000..a64bb3a02
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDateMilliProducer.java
@@ -0,0 +1,49 @@
+/*
+ * 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.DateMilliVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that converts days in milliseconds from a {@link DateMilliVector} and produces date
+ * (INT) values, writes data to an Avro encoder.
+ */
+public class AvroDateMilliProducer extends BaseAvroProducer {
+
+ // Convert milliseconds to days for Avro date type
+
+ private static final long MILLIS_PER_DAY = 86400000;
+
+ /** Instantiate an AvroDateMilliProducer. */
+ public AvroDateMilliProducer(DateMilliVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long millis = vector.getDataBuffer().getLong(currentIndex * (long) DateMilliVector.TYPE_WIDTH);
+ long days = millis / MILLIS_PER_DAY;
+ if (days > (long) Integer.MAX_VALUE || days < (long) Integer.MIN_VALUE) {
+ throw new ArithmeticException("Date value is too large for Avro encoding");
+ }
+ encoder.writeInt((int) days);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimal256Producer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimal256Producer.java
new file mode 100644
index 000000000..f72aa6d9e
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimal256Producer.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 java.math.BigDecimal;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.vector.Decimal256Vector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces decimal values from a {@link Decimal256Vector}, writes data to an Avro
+ * encoder.
+ */
+public class AvroDecimal256Producer extends BaseAvroProducer {
+
+ // Logic is the same as for DecimalVector (128 bit)
+
+ byte[] encodedBytes = new byte[Decimal256Vector.TYPE_WIDTH];
+
+ /** Instantiate an AvroDecimalProducer. */
+ public AvroDecimal256Producer(Decimal256Vector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ BigDecimal value = vector.getObject(currentIndex++);
+ AvroDecimalProducer.encodeDecimal(value, encodedBytes);
+ encoder.writeFixed(encodedBytes);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimalProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimalProducer.java
new file mode 100644
index 000000000..51ad7c720
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroDecimalProducer.java
@@ -0,0 +1,59 @@
+/*
+ * 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.math.BigDecimal;
+import org.apache.arrow.adapter.avro.producers.BaseAvroProducer;
+import org.apache.arrow.vector.DecimalVector;
+import org.apache.arrow.vector.util.DecimalUtility;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that produces decimal values from a {@link DecimalVector}, writes data to an Avro
+ * encoder.
+ */
+public class AvroDecimalProducer extends BaseAvroProducer {
+
+ // Arrow stores decimals with native endianness, but Avro requires big endian
+ // Writing the Arrow representation as fixed bytes fails on little-end machines
+ // Instead, we replicate the big endian logic explicitly here
+ // See DecimalUtility.writeByteArrayToArrowBufHelper
+
+ byte[] encodedBytes = new byte[DecimalVector.TYPE_WIDTH];
+
+ /** Instantiate an AvroDecimalProducer. */
+ public AvroDecimalProducer(DecimalVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ // Use getObject() to go back to a BigDecimal then re-encode
+ BigDecimal value = vector.getObject(currentIndex++);
+ encodeDecimal(value, encodedBytes);
+ encoder.writeFixed(encodedBytes);
+ }
+
+ static void encodeDecimal(BigDecimal value, byte[] encodedBytes) {
+ byte[] valueBytes = value.unscaledValue().toByteArray();
+ byte[] padding = valueBytes[0] < 0 ? DecimalUtility.minus_one : DecimalUtility.zeroes;
+ System.arraycopy(padding, 0, encodedBytes, 0, encodedBytes.length - valueBytes.length);
+ System.arraycopy(
+ valueBytes, 0, encodedBytes, encodedBytes.length - valueBytes.length, valueBytes.length);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMicroProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMicroProducer.java
new file mode 100644
index 000000000..203d10203
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMicroProducer.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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeMicroVector;
+
+/**
+ * Producer that produces time (microseconds) values from a {@link TimeMicroVector}, writes data to
+ * an Avro encoder.
+ */
+public class AvroTimeMicroProducer extends AvroBigIntProducer {
+
+ // Time in microseconds stored as long, matches Avro time-micros type
+
+ /** Instantiate an AvroTimeMicroProducer. */
+ public AvroTimeMicroProducer(TimeMicroVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMilliProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeMilliProducer.java
new file mode 100644
index 000000000..2a452e75a
--- /dev/null
+++ b/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
+
+ /** Instantiate an AvroTimeMilliProducer. */
+ public AvroTimeMilliProducer(TimeMilliVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeNanoProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeNanoProducer.java
new file mode 100644
index 000000000..7034dbbb5
--- /dev/null
+++ b/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 {
+
+ // 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;
+ encoder.writeLong(micros);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeSecProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeSecProducer.java
new file mode 100644
index 000000000..951605b6c
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimeSecProducer.java
@@ -0,0 +1,52 @@
+/*
+ * 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.TimeSecVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that converts seconds from a {@link TimeSecVector} and produces time (microseconds)
+ * values, writes data to an Avro encoder.
+ */
+public class AvroTimeSecProducer extends BaseAvroProducer {
+
+ // Convert seconds to milliseconds for Avro time-millis (INT) type
+ // INT is enough to cover the number of milliseconds in a day
+ // So overflows should not happen if values are valid times of day
+
+ private static final int MILLIS_PER_SECOND = 1000;
+ private static final long OVERFLOW_LIMIT = Integer.MAX_VALUE / 1000;
+
+ /** Instantiate an AvroTimeSecProducer. */
+ public AvroTimeSecProducer(TimeSecVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ int seconds = vector.getDataBuffer().getInt(currentIndex * (long) TimeSecVector.TYPE_WIDTH);
+ if (Math.abs(seconds) > OVERFLOW_LIMIT) {
+ throw new ArithmeticException("Time value is too large for Avro encoding");
+ }
+ int millis = seconds * MILLIS_PER_SECOND;
+ encoder.writeInt(millis);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroProducer.java
new file mode 100644
index 000000000..4e744b5e7
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroProducer.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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampMicroVector;
+
+/**
+ * Producer that produces local timestamp (microseconds) values from a {@link TimeStampMicroVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampMicroProducer extends AvroBigIntProducer {
+
+ // Local timestamp in epoch microseconds stored as long, matches Avro local-timestamp-micros type
+
+ /** Instantiate an AvroTimestampMicroProducer. */
+ public AvroTimestampMicroProducer(TimeStampMicroVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroTzProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroTzProducer.java
new file mode 100644
index 000000000..ece748230
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMicroTzProducer.java
@@ -0,0 +1,35 @@
+/*
+ * 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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampMicroTZVector;
+
+/**
+ * Producer that produces UTC timestamp (microseconds) values from a {@link TimeStampMicroTZVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampMicroTzProducer extends AvroBigIntProducer {
+
+ // UTC timestamp in epoch microseconds stored as long, matches Avro timestamp-micros type
+ // Both Arrow and Avro store zone-aware times in UTC so zone conversion is not needed
+
+ /** Instantiate an AvroTimestampMicroTzProducer. */
+ public AvroTimestampMicroTzProducer(TimeStampMicroTZVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliProducer.java
new file mode 100644
index 000000000..e71acff22
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliProducer.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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampMilliVector;
+
+/**
+ * Producer that produces local timestamp (milliseconds) values from a {@link TimeStampMilliVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampMilliProducer extends AvroBigIntProducer {
+
+ // Local timestamp in epoch milliseconds stored as long, matches Avro local-timestamp-millis type
+
+ /** Instantiate an AvroTimestampMilliProducer. */
+ public AvroTimestampMilliProducer(TimeStampMilliVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliTzProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliTzProducer.java
new file mode 100644
index 000000000..b1b55fca7
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampMilliTzProducer.java
@@ -0,0 +1,35 @@
+/*
+ * 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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampMilliTZVector;
+
+/**
+ * Producer that produces UTC timestamp (milliseconds) values from a {@link TimeStampMilliTZVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampMilliTzProducer extends AvroBigIntProducer {
+
+ // UTC timestamp in epoch milliseconds stored as long, matches Avro timestamp-millis type
+ // Both Arrow and Avro store zone-aware times in UTC so zone conversion is not needed
+
+ /** Instantiate an AvroTimestampMilliTzProducer. */
+ public AvroTimestampMilliTzProducer(TimeStampMilliTZVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoProducer.java
new file mode 100644
index 000000000..9e172ea91
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoProducer.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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampNanoVector;
+
+/**
+ * Producer that produces local timestamp (nanoseconds) values from a {@link TimeStampNanoVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampNanoProducer extends AvroBigIntProducer {
+
+ // Local timestamp in epoch nanoseconds stored as long, matches Avro local-timestamp-nanos type
+
+ /** Instantiate an AvroTimestampNanoProducer. */
+ public AvroTimestampNanoProducer(TimeStampNanoVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoTzProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoTzProducer.java
new file mode 100644
index 000000000..ae261d839
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampNanoTzProducer.java
@@ -0,0 +1,35 @@
+/*
+ * 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.AvroBigIntProducer;
+import org.apache.arrow.vector.TimeStampNanoTZVector;
+
+/**
+ * Producer that produces local timestamp (nanoseconds) values from a {@link TimeStampNanoTZVector},
+ * writes data to an Avro encoder.
+ */
+public class AvroTimestampNanoTzProducer extends AvroBigIntProducer {
+
+ // UTC timestamp in epoch nanoseconds stored as long, matches Avro timestamp-nanos type
+ // Both Arrow and Avro store zone-aware times in UTC so zone conversion is not needed
+
+ /** Instantiate an AvroTimestampNanoTzProducer. */
+ public AvroTimestampNanoTzProducer(TimeStampNanoTZVector vector) {
+ super(vector);
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecProducer.java
new file mode 100644
index 000000000..a6ade2d19
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecProducer.java
@@ -0,0 +1,52 @@
+/*
+ * 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.TimeStampSecVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that converts epoch seconds from a {@link TimeStampSecVector} and produces local
+ * timestamp (milliseconds) values, writes data to an Avro encoder.
+ */
+public class AvroTimestampSecProducer extends BaseAvroProducer {
+
+ // Avro does not support timestamps in seconds, so convert to local-timestamp-millis type
+ // Check for overflow and raise an exception
+
+ private static final long MILLIS_PER_SECOND = 1000;
+ private static final long OVERFLOW_LIMIT = Long.MAX_VALUE / MILLIS_PER_SECOND;
+
+ /** Instantiate an AvroTimestampSecProducer. */
+ public AvroTimestampSecProducer(TimeStampSecVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long seconds =
+ vector.getDataBuffer().getLong(currentIndex * (long) TimeStampSecVector.TYPE_WIDTH);
+ if (Math.abs(seconds) > OVERFLOW_LIMIT) {
+ throw new ArithmeticException("Timestamp value is too large for Avro encoding");
+ }
+ long millis = seconds * MILLIS_PER_SECOND;
+ encoder.writeLong(millis);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecTzProducer.java b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecTzProducer.java
new file mode 100644
index 000000000..bd6cc14da
--- /dev/null
+++ b/adapter/avro/src/main/java/org/apache/arrow/adapter/avro/producers/logical/AvroTimestampSecTzProducer.java
@@ -0,0 +1,55 @@
+/*
+ * 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.TimeStampSecTZVector;
+import org.apache.arrow.vector.TimeStampVector;
+import org.apache.avro.io.Encoder;
+
+/**
+ * Producer that converts epoch seconds from a {@link TimeStampSecTZVector} and produces UTC
+ * timestamp (milliseconds) values, writes data to an Avro encoder.
+ */
+public class AvroTimestampSecTzProducer extends BaseAvroProducer {
+
+ // Avro does not support timestamps in seconds, so convert to timestamp-millis type
+ // Check for overflow and raise an exception
+
+ // Both Arrow and Avro store zone-aware times in UTC so zone conversion is not needed
+
+ private static final long MILLIS_PER_SECOND = 1000;
+ private static final long OVERFLOW_LIMIT = Long.MAX_VALUE / MILLIS_PER_SECOND;
+
+ /** Instantiate an AvroTimestampSecTzProducer. */
+ public AvroTimestampSecTzProducer(TimeStampSecTZVector vector) {
+ super(vector);
+ }
+
+ @Override
+ public void produce(Encoder encoder) throws IOException {
+ long utcSeconds =
+ vector.getDataBuffer().getLong(currentIndex * (long) TimeStampVector.TYPE_WIDTH);
+ if (Math.abs(utcSeconds) > OVERFLOW_LIMIT) {
+ throw new ArithmeticException("Timestamp value is too large for Avro encoding");
+ }
+ long utcMillis = utcSeconds * MILLIS_PER_SECOND;
+ encoder.writeLong(utcMillis);
+ currentIndex++;
+ }
+}
diff --git a/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroDataTest.java b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroDataTest.java
new file mode 100644
index 000000000..2d70b4502
--- /dev/null
+++ b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroDataTest.java
@@ -0,0 +1,2820 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.*;
+
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.io.InputStream;
+import java.math.BigDecimal;
+import java.math.RoundingMode;
+import java.nio.ByteBuffer;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.ZonedDateTime;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.arrow.adapter.avro.producers.CompositeAvroProducer;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.memory.util.Float16;
+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.VectorSchemaRoot;
+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.writer.BaseWriter;
+import org.apache.arrow.vector.complex.writer.FieldWriter;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.arrow.vector.util.JsonStringArrayList;
+import org.apache.arrow.vector.util.JsonStringHashMap;
+import org.apache.avro.Conversions;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.apache.avro.generic.GenericData;
+import org.apache.avro.generic.GenericDatumReader;
+import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
+import org.apache.avro.util.Utf8;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+
+public class ArrowToAvroDataTest {
+
+ @TempDir public static File TMP;
+
+ // Data production for primitive types, nullable and non-nullable
+
+ @Test
+ public void testWriteNullColumn() throws Exception {
+
+ // Field definition
+ FieldType nullField = new FieldType(false, new ArrowType.Null(), null);
+
+ // Create empty vector
+ NullVector nullVector = new NullVector(new Field("nullColumn", nullField, null));
+
+ int rowCount = 10;
+
+ // Set up VSR
+ List vectors = Arrays.asList(nullVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set all values to null
+ for (int row = 0; row < rowCount; row++) {
+ nullVector.setNull(row);
+ }
+
+ File dataFile = new File(TMP, "testWriteNullColumn.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertNull(record.get("nullColumn"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteBooleans() throws Exception {
+
+ // Field definition
+ FieldType booleanField = new FieldType(false, new ArrowType.Bool(), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ BitVector booleanVector = new BitVector(new Field("boolean", booleanField, null), allocator);
+
+ // Set up VSR
+ List vectors = Arrays.asList(booleanVector);
+ int rowCount = 10;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ for (int row = 0; row < rowCount; row++) {
+ booleanVector.set(row, row % 2 == 0 ? 1 : 0);
+ }
+
+ File dataFile = new File(TMP, "testWriteBooleans.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(booleanVector.get(row) == 1, record.get("boolean"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableBooleans() throws Exception {
+
+ // Field definition
+ FieldType booleanField = new FieldType(true, new ArrowType.Bool(), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ BitVector booleanVector = new BitVector(new Field("boolean", booleanField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors = Arrays.asList(booleanVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Null value
+ booleanVector.setNull(0);
+
+ // False value
+ booleanVector.set(1, 0);
+
+ // True value
+ booleanVector.set(2, 1);
+
+ File dataFile = new File(TMP, "testWriteNullableBooleans.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("boolean"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(booleanVector.get(row) == 1, record.get("boolean"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteIntegers() throws Exception {
+
+ // Field definitions
+ FieldType int8Field = new FieldType(false, new ArrowType.Int(8, true), null);
+ FieldType int16Field = new FieldType(false, new ArrowType.Int(16, true), null);
+ FieldType int32Field = new FieldType(false, new ArrowType.Int(32, true), null);
+ FieldType int64Field = new FieldType(false, new ArrowType.Int(64, true), null);
+ FieldType uint8Field = new FieldType(false, new ArrowType.Int(8, false), null);
+ FieldType uint16Field = new FieldType(false, new ArrowType.Int(16, false), null);
+ FieldType uint32Field = new FieldType(false, new ArrowType.Int(32, false), null);
+ FieldType uint64Field = new FieldType(false, new ArrowType.Int(64, false), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TinyIntVector int8Vector = new TinyIntVector(new Field("int8", int8Field, null), allocator);
+ SmallIntVector int16Vector =
+ new SmallIntVector(new Field("int16", int16Field, null), allocator);
+ IntVector int32Vector = new IntVector(new Field("int32", int32Field, null), allocator);
+ BigIntVector int64Vector = new BigIntVector(new Field("int64", int64Field, null), allocator);
+ UInt1Vector uint8Vector = new UInt1Vector(new Field("uint8", uint8Field, null), allocator);
+ UInt2Vector uint16Vector = new UInt2Vector(new Field("uint16", uint16Field, null), allocator);
+ UInt4Vector uint32Vector = new UInt4Vector(new Field("uint32", uint32Field, null), allocator);
+ UInt8Vector uint64Vector = new UInt8Vector(new Field("uint64", uint64Field, null), allocator);
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ int8Vector,
+ int16Vector,
+ int32Vector,
+ int64Vector,
+ uint8Vector,
+ uint16Vector,
+ uint32Vector,
+ uint64Vector);
+
+ int rowCount = 12;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ for (int row = 0; row < 10; row++) {
+ int8Vector.set(row, 11 * row * (row % 2 == 0 ? 1 : -1));
+ int16Vector.set(row, 63 * row * (row % 2 == 0 ? 1 : -1));
+ int32Vector.set(row, 513 * row * (row % 2 == 0 ? 1 : -1));
+ int64Vector.set(row, 3791L * row * (row % 2 == 0 ? 1 : -1));
+ uint8Vector.set(row, 11 * row);
+ uint16Vector.set(row, 63 * row);
+ uint32Vector.set(row, 513 * row);
+ uint64Vector.set(row, 3791L * row);
+ }
+
+ // Min values
+ int8Vector.set(10, Byte.MIN_VALUE);
+ int16Vector.set(10, Short.MIN_VALUE);
+ int32Vector.set(10, Integer.MIN_VALUE);
+ int64Vector.set(10, Long.MIN_VALUE);
+ uint8Vector.set(10, 0);
+ uint16Vector.set(10, 0);
+ uint32Vector.set(10, 0);
+ uint64Vector.set(10, 0);
+
+ // Max values
+ int8Vector.set(11, Byte.MAX_VALUE);
+ int16Vector.set(11, Short.MAX_VALUE);
+ int32Vector.set(11, Integer.MAX_VALUE);
+ int64Vector.set(11, Long.MAX_VALUE);
+ uint8Vector.set(11, 0xff);
+ uint16Vector.set(11, 0xffff);
+ uint32Vector.set(11, 0xffffffff);
+ uint64Vector.set(11, Long.MAX_VALUE); // Max that can be encoded
+
+ File dataFile = new File(TMP, "testWriteIntegers.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals((int) int8Vector.get(row), record.get("int8"));
+ assertEquals((int) int16Vector.get(row), record.get("int16"));
+ assertEquals(int32Vector.get(row), record.get("int32"));
+ assertEquals(int64Vector.get(row), record.get("int64"));
+ assertEquals(Byte.toUnsignedInt(uint8Vector.get(row)), record.get("uint8"));
+ assertEquals(Short.toUnsignedInt((short) uint16Vector.get(row)), record.get("uint16"));
+ assertEquals(Integer.toUnsignedLong(uint32Vector.get(row)), record.get("uint32"));
+ assertEquals(uint64Vector.get(row), record.get("uint64"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableIntegers() throws Exception {
+
+ // Field definitions
+ FieldType int8Field = new FieldType(true, new ArrowType.Int(8, true), null);
+ FieldType int16Field = new FieldType(true, new ArrowType.Int(16, true), null);
+ FieldType int32Field = new FieldType(true, new ArrowType.Int(32, true), null);
+ FieldType int64Field = new FieldType(true, new ArrowType.Int(64, true), null);
+ FieldType uint8Field = new FieldType(true, new ArrowType.Int(8, false), null);
+ FieldType uint16Field = new FieldType(true, new ArrowType.Int(16, false), null);
+ FieldType uint32Field = new FieldType(true, new ArrowType.Int(32, false), null);
+ FieldType uint64Field = new FieldType(true, new ArrowType.Int(64, false), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TinyIntVector int8Vector = new TinyIntVector(new Field("int8", int8Field, null), allocator);
+ SmallIntVector int16Vector =
+ new SmallIntVector(new Field("int16", int16Field, null), allocator);
+ IntVector int32Vector = new IntVector(new Field("int32", int32Field, null), allocator);
+ BigIntVector int64Vector = new BigIntVector(new Field("int64", int64Field, null), allocator);
+ UInt1Vector uint8Vector = new UInt1Vector(new Field("uint8", uint8Field, null), allocator);
+ UInt2Vector uint16Vector = new UInt2Vector(new Field("uint16", uint16Field, null), allocator);
+ UInt4Vector uint32Vector = new UInt4Vector(new Field("uint32", uint32Field, null), allocator);
+ UInt8Vector uint64Vector = new UInt8Vector(new Field("uint64", uint64Field, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ int8Vector,
+ int16Vector,
+ int32Vector,
+ int64Vector,
+ uint8Vector,
+ uint16Vector,
+ uint32Vector,
+ uint64Vector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Null values
+ int8Vector.setNull(0);
+ int16Vector.setNull(0);
+ int32Vector.setNull(0);
+ int64Vector.setNull(0);
+ uint8Vector.setNull(0);
+ uint16Vector.setNull(0);
+ uint32Vector.setNull(0);
+ uint64Vector.setNull(0);
+
+ // Zero values
+ int8Vector.set(1, 0);
+ int16Vector.set(1, 0);
+ int32Vector.set(1, 0);
+ int64Vector.set(1, 0);
+ uint8Vector.set(1, 0);
+ uint16Vector.set(1, 0);
+ uint32Vector.set(1, 0);
+ uint64Vector.set(1, 0);
+
+ // Non-zero values
+ int8Vector.set(2, Byte.MAX_VALUE);
+ int16Vector.set(2, Short.MAX_VALUE);
+ int32Vector.set(2, Integer.MAX_VALUE);
+ int64Vector.set(2, Long.MAX_VALUE);
+ uint8Vector.set(2, Byte.MAX_VALUE);
+ uint16Vector.set(2, Short.MAX_VALUE);
+ uint32Vector.set(2, Integer.MAX_VALUE);
+ uint64Vector.set(2, Long.MAX_VALUE);
+
+ File dataFile = new File(TMP, "testWriteNullableIntegers.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("int8"));
+ assertNull(record.get("int16"));
+ assertNull(record.get("int32"));
+ assertNull(record.get("int64"));
+ assertNull(record.get("uint8"));
+ assertNull(record.get("uint16"));
+ assertNull(record.get("uint32"));
+ assertNull(record.get("uint64"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals((int) int8Vector.get(row), record.get("int8"));
+ assertEquals((int) int16Vector.get(row), record.get("int16"));
+ assertEquals(int32Vector.get(row), record.get("int32"));
+ assertEquals(int64Vector.get(row), record.get("int64"));
+ assertEquals(Byte.toUnsignedInt(uint8Vector.get(row)), record.get("uint8"));
+ assertEquals(Short.toUnsignedInt((short) uint16Vector.get(row)), record.get("uint16"));
+ assertEquals(Integer.toUnsignedLong(uint32Vector.get(row)), record.get("uint32"));
+ assertEquals(uint64Vector.get(row), record.get("uint64"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteFloatingPoints() throws Exception {
+
+ // Field definitions
+ FieldType float16Field =
+ new FieldType(false, new ArrowType.FloatingPoint(FloatingPointPrecision.HALF), null);
+ FieldType float32Field =
+ new FieldType(false, new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), null);
+ FieldType float64Field =
+ new FieldType(false, new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ Float2Vector float16Vector =
+ new Float2Vector(new Field("float16", float16Field, null), allocator);
+ Float4Vector float32Vector =
+ new Float4Vector(new Field("float32", float32Field, null), allocator);
+ Float8Vector float64Vector =
+ new Float8Vector(new Field("float64", float64Field, null), allocator);
+
+ // Set up VSR
+ List vectors = Arrays.asList(float16Vector, float32Vector, float64Vector);
+ int rowCount = 15;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ for (int row = 0; row < 10; row++) {
+ float16Vector.set(row, Float16.toFloat16(3.6f * row * (row % 2 == 0 ? 1.0f : -1.0f)));
+ float32Vector.set(row, 37.6f * row * (row % 2 == 0 ? 1 : -1));
+ float64Vector.set(row, 37.6d * row * (row % 2 == 0 ? 1 : -1));
+ }
+
+ float16Vector.set(10, Float16.toFloat16(Float.MIN_VALUE));
+ float32Vector.set(10, Float.MIN_VALUE);
+ float64Vector.set(10, Double.MIN_VALUE);
+
+ float16Vector.set(11, Float16.toFloat16(Float.MAX_VALUE));
+ float32Vector.set(11, Float.MAX_VALUE);
+ float64Vector.set(11, Double.MAX_VALUE);
+
+ float16Vector.set(12, Float16.toFloat16(Float.NaN));
+ float32Vector.set(12, Float.NaN);
+ float64Vector.set(12, Double.NaN);
+
+ float16Vector.set(13, Float16.toFloat16(Float.POSITIVE_INFINITY));
+ float32Vector.set(13, Float.POSITIVE_INFINITY);
+ float64Vector.set(13, Double.POSITIVE_INFINITY);
+
+ float16Vector.set(14, Float16.toFloat16(Float.NEGATIVE_INFINITY));
+ float32Vector.set(14, Float.NEGATIVE_INFINITY);
+ float64Vector.set(14, Double.NEGATIVE_INFINITY);
+
+ File dataFile = new File(TMP, "testWriteFloatingPoints.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(float16Vector.getValueAsFloat(row), record.get("float16"));
+ assertEquals(float32Vector.get(row), record.get("float32"));
+ assertEquals(float64Vector.get(row), record.get("float64"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableFloatingPoints() throws Exception {
+
+ // Field definitions
+ FieldType float16Field =
+ new FieldType(true, new ArrowType.FloatingPoint(FloatingPointPrecision.HALF), null);
+ FieldType float32Field =
+ new FieldType(true, new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE), null);
+ FieldType float64Field =
+ new FieldType(true, new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ Float2Vector float16Vector =
+ new Float2Vector(new Field("float16", float16Field, null), allocator);
+ Float4Vector float32Vector =
+ new Float4Vector(new Field("float32", float32Field, null), allocator);
+ Float8Vector float64Vector =
+ new Float8Vector(new Field("float64", float64Field, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors = Arrays.asList(float16Vector, float32Vector, float64Vector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Null values
+ float16Vector.setNull(0);
+ float32Vector.setNull(0);
+ float64Vector.setNull(0);
+
+ // Zero values
+ float16Vector.setSafeWithPossibleTruncate(1, 0.0f);
+ float32Vector.set(1, 0.0f);
+ float64Vector.set(1, 0.0);
+
+ // Non-zero values
+ float16Vector.setSafeWithPossibleTruncate(2, 1.0f);
+ float32Vector.set(2, 1.0f);
+ float64Vector.set(2, 1.0);
+
+ File dataFile = new File(TMP, "testWriteNullableFloatingPoints.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("float16"));
+ assertNull(record.get("float32"));
+ assertNull(record.get("float64"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(float16Vector.getValueAsFloat(row), record.get("float16"));
+ assertEquals(float32Vector.get(row), record.get("float32"));
+ assertEquals(float64Vector.get(row), record.get("float64"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteStrings() throws Exception {
+
+ // Field definition
+ FieldType stringField = new FieldType(false, new ArrowType.Utf8(), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ VarCharVector stringVector =
+ new VarCharVector(new Field("string", stringField, null), allocator);
+
+ // Set up VSR
+ List vectors = Arrays.asList(stringVector);
+ int rowCount = 5;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ stringVector.setSafe(0, "Hello world!".getBytes());
+ stringVector.setSafe(1, "<%**\r\n\t\\abc\0$$>".getBytes());
+ stringVector.setSafe(2, "ä½ å¥½ä¸–ç•Œ".getBytes());
+ stringVector.setSafe(3, "Ù…Ø±ØØ¨Ø§ بالعالم".getBytes());
+ stringVector.setSafe(4, "(P ∧ P ⇒ Q) ⇒ Q".getBytes());
+
+ File dataFile = new File(TMP, "testWriteStrings.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(stringVector.getObject(row).toString(), record.get("string").toString());
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableStrings() throws Exception {
+
+ // Field definition
+ FieldType stringField = new FieldType(true, new ArrowType.Utf8(), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ VarCharVector stringVector =
+ new VarCharVector(new Field("string", stringField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors = Arrays.asList(stringVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ stringVector.setNull(0);
+ stringVector.setSafe(1, "".getBytes());
+ stringVector.setSafe(2, "not empty".getBytes());
+
+ File dataFile = new File(TMP, "testWriteNullableStrings.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("string"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(stringVector.getObject(row).toString(), record.get("string").toString());
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteBinary() throws Exception {
+
+ // Field definition
+ FieldType binaryField = new FieldType(false, new ArrowType.Binary(), null);
+ FieldType fixedField = new FieldType(false, new ArrowType.FixedSizeBinary(5), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ VarBinaryVector binaryVector =
+ new VarBinaryVector(new Field("binary", binaryField, null), allocator);
+ FixedSizeBinaryVector fixedVector =
+ new FixedSizeBinaryVector(new Field("fixed", fixedField, null), allocator);
+
+ // Set up VSR
+ List vectors = Arrays.asList(binaryVector, fixedVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ binaryVector.setSafe(0, new byte[] {1, 2, 3});
+ binaryVector.setSafe(1, new byte[] {4, 5, 6, 7});
+ binaryVector.setSafe(2, new byte[] {8, 9});
+
+ fixedVector.setSafe(0, new byte[] {1, 2, 3, 4, 5});
+ fixedVector.setSafe(1, new byte[] {4, 5, 6, 7, 8, 9});
+ fixedVector.setSafe(2, new byte[] {8, 9, 10, 11, 12});
+
+ File dataFile = new File(TMP, "testWriteBinary.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ ByteBuffer buf = ((ByteBuffer) record.get("binary"));
+ byte[] bytes = new byte[buf.remaining()];
+ buf.get(bytes);
+ byte[] fixedBytes = ((GenericData.Fixed) record.get("fixed")).bytes();
+ assertArrayEquals(binaryVector.getObject(row), bytes);
+ assertArrayEquals(fixedVector.getObject(row), fixedBytes);
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableBinary() throws Exception {
+
+ // Field definition
+ FieldType binaryField = new FieldType(true, new ArrowType.Binary(), null);
+ FieldType fixedField = new FieldType(true, new ArrowType.FixedSizeBinary(5), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ VarBinaryVector binaryVector =
+ new VarBinaryVector(new Field("binary", binaryField, null), allocator);
+ FixedSizeBinaryVector fixedVector =
+ new FixedSizeBinaryVector(new Field("fixed", fixedField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors = Arrays.asList(binaryVector, fixedVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ binaryVector.setNull(0);
+ binaryVector.setSafe(1, new byte[] {});
+ binaryVector.setSafe(2, new byte[] {10, 11, 12});
+
+ fixedVector.setNull(0);
+ fixedVector.setSafe(1, new byte[] {0, 0, 0, 0, 0});
+ fixedVector.setSafe(2, new byte[] {10, 11, 12, 13, 14});
+
+ File dataFile = new File(TMP, "testWriteNullableBinary.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("binary"));
+ assertNull(record.get("fixed"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ ByteBuffer buf = ((ByteBuffer) record.get("binary"));
+ byte[] bytes = new byte[buf.remaining()];
+ buf.get(bytes);
+ byte[] fixedBytes = ((GenericData.Fixed) record.get("fixed")).bytes();
+ assertArrayEquals(binaryVector.getObject(row), bytes);
+ assertArrayEquals(fixedVector.getObject(row), fixedBytes);
+ }
+ }
+ }
+ }
+
+ // Data production for logical types, nullable and non-nullable
+
+ @Test
+ public void testWriteDecimals() throws Exception {
+
+ // Field definitions
+ FieldType decimal128Field1 = new FieldType(false, new ArrowType.Decimal(38, 10, 128), null);
+ FieldType decimal128Field2 = new FieldType(false, new ArrowType.Decimal(38, 5, 128), null);
+ FieldType decimal256Field1 = new FieldType(false, new ArrowType.Decimal(76, 20, 256), null);
+ FieldType decimal256Field2 = new FieldType(false, new ArrowType.Decimal(76, 10, 256), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ DecimalVector decimal128Vector1 =
+ new DecimalVector(new Field("decimal128_1", decimal128Field1, null), allocator);
+ DecimalVector decimal128Vector2 =
+ new DecimalVector(new Field("decimal128_2", decimal128Field2, null), allocator);
+ Decimal256Vector decimal256Vector1 =
+ new Decimal256Vector(new Field("decimal256_1", decimal256Field1, null), allocator);
+ Decimal256Vector decimal256Vector2 =
+ new Decimal256Vector(new Field("decimal256_2", decimal256Field2, null), allocator);
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(decimal128Vector1, decimal128Vector2, decimal256Vector1, decimal256Vector2);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ decimal128Vector1.setSafe(
+ 0, new BigDecimal("12345.67890").setScale(10, RoundingMode.UNNECESSARY));
+ decimal128Vector1.setSafe(
+ 1, new BigDecimal("-98765.43210").setScale(10, RoundingMode.UNNECESSARY));
+ decimal128Vector1.setSafe(
+ 2, new BigDecimal("54321.09876").setScale(10, RoundingMode.UNNECESSARY));
+
+ decimal128Vector2.setSafe(
+ 0, new BigDecimal("12345.67890").setScale(5, RoundingMode.UNNECESSARY));
+ decimal128Vector2.setSafe(
+ 1, new BigDecimal("-98765.43210").setScale(5, RoundingMode.UNNECESSARY));
+ decimal128Vector2.setSafe(
+ 2, new BigDecimal("54321.09876").setScale(5, RoundingMode.UNNECESSARY));
+
+ decimal256Vector1.setSafe(
+ 0,
+ new BigDecimal("12345678901234567890.12345678901234567890")
+ .setScale(20, RoundingMode.UNNECESSARY));
+ decimal256Vector1.setSafe(
+ 1,
+ new BigDecimal("-98765432109876543210.98765432109876543210")
+ .setScale(20, RoundingMode.UNNECESSARY));
+ decimal256Vector1.setSafe(
+ 2,
+ new BigDecimal("54321098765432109876.54321098765432109876")
+ .setScale(20, RoundingMode.UNNECESSARY));
+
+ decimal256Vector2.setSafe(
+ 0,
+ new BigDecimal("12345678901234567890.1234567890").setScale(10, RoundingMode.UNNECESSARY));
+ decimal256Vector2.setSafe(
+ 1,
+ new BigDecimal("-98765432109876543210.9876543210")
+ .setScale(10, RoundingMode.UNNECESSARY));
+ decimal256Vector2.setSafe(
+ 2,
+ new BigDecimal("54321098765432109876.5432109876").setScale(10, RoundingMode.UNNECESSARY));
+
+ File dataFile = new File(TMP, "testWriteDecimals.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ decimal128Vector1.getObject(row), decodeFixedDecimal(record, "decimal128_1"));
+ assertEquals(
+ decimal128Vector2.getObject(row), decodeFixedDecimal(record, "decimal128_2"));
+ assertEquals(
+ decimal256Vector1.getObject(row), decodeFixedDecimal(record, "decimal256_1"));
+ assertEquals(
+ decimal256Vector2.getObject(row), decodeFixedDecimal(record, "decimal256_2"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableDecimals() throws Exception {
+
+ // Field definitions
+ FieldType decimal128Field1 = new FieldType(true, new ArrowType.Decimal(38, 10, 128), null);
+ FieldType decimal128Field2 = new FieldType(true, new ArrowType.Decimal(38, 5, 128), null);
+ FieldType decimal256Field1 = new FieldType(true, new ArrowType.Decimal(76, 20, 256), null);
+ FieldType decimal256Field2 = new FieldType(true, new ArrowType.Decimal(76, 10, 256), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ DecimalVector decimal128Vector1 =
+ new DecimalVector(new Field("decimal128_1", decimal128Field1, null), allocator);
+ DecimalVector decimal128Vector2 =
+ new DecimalVector(new Field("decimal128_2", decimal128Field2, null), allocator);
+ Decimal256Vector decimal256Vector1 =
+ new Decimal256Vector(new Field("decimal256_1", decimal256Field1, null), allocator);
+ Decimal256Vector decimal256Vector2 =
+ new Decimal256Vector(new Field("decimal256_2", decimal256Field2, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(decimal128Vector1, decimal128Vector2, decimal256Vector1, decimal256Vector2);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ decimal128Vector1.setNull(0);
+ decimal128Vector1.setSafe(1, BigDecimal.ZERO.setScale(10, RoundingMode.UNNECESSARY));
+ decimal128Vector1.setSafe(
+ 2, new BigDecimal("12345.67890").setScale(10, RoundingMode.UNNECESSARY));
+
+ decimal128Vector2.setNull(0);
+ decimal128Vector2.setSafe(1, BigDecimal.ZERO.setScale(5, RoundingMode.UNNECESSARY));
+ decimal128Vector2.setSafe(
+ 2, new BigDecimal("98765.43210").setScale(5, RoundingMode.UNNECESSARY));
+
+ decimal256Vector1.setNull(0);
+ decimal256Vector1.setSafe(1, BigDecimal.ZERO.setScale(20, RoundingMode.UNNECESSARY));
+ decimal256Vector1.setSafe(
+ 2,
+ new BigDecimal("12345678901234567890.12345678901234567890")
+ .setScale(20, RoundingMode.UNNECESSARY));
+
+ decimal256Vector2.setNull(0);
+ decimal256Vector2.setSafe(1, BigDecimal.ZERO.setScale(10, RoundingMode.UNNECESSARY));
+ decimal256Vector2.setSafe(
+ 2,
+ new BigDecimal("98765432109876543210.9876543210").setScale(10, RoundingMode.UNNECESSARY));
+
+ File dataFile = new File(TMP, "testWriteNullableDecimals.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("decimal128_1"));
+ assertNull(record.get("decimal128_2"));
+ assertNull(record.get("decimal256_1"));
+ assertNull(record.get("decimal256_2"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ decimal128Vector1.getObject(row), decodeFixedDecimal(record, "decimal128_1"));
+ assertEquals(
+ decimal128Vector2.getObject(row), decodeFixedDecimal(record, "decimal128_2"));
+ assertEquals(
+ decimal256Vector1.getObject(row), decodeFixedDecimal(record, "decimal256_1"));
+ assertEquals(
+ decimal256Vector2.getObject(row), decodeFixedDecimal(record, "decimal256_2"));
+ }
+ }
+ }
+ }
+
+ private static BigDecimal decodeFixedDecimal(GenericRecord record, String fieldName) {
+ GenericData.Fixed fixed = (GenericData.Fixed) record.get(fieldName);
+ var logicalType = LogicalTypes.fromSchema(fixed.getSchema());
+ return new Conversions.DecimalConversion().fromFixed(fixed, fixed.getSchema(), logicalType);
+ }
+
+ @Test
+ public void testWriteDates() throws Exception {
+
+ // Field definitions
+ FieldType dateDayField = new FieldType(false, new ArrowType.Date(DateUnit.DAY), null);
+ FieldType dateMillisField =
+ new FieldType(false, new ArrowType.Date(DateUnit.MILLISECOND), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ DateDayVector dateDayVector =
+ new DateDayVector(new Field("dateDay", dateDayField, null), allocator);
+ DateMilliVector dateMillisVector =
+ new DateMilliVector(new Field("dateMillis", dateMillisField, null), allocator);
+
+ // Set up VSR
+ List vectors = Arrays.asList(dateDayVector, dateMillisVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ dateDayVector.setSafe(0, (int) LocalDate.now().toEpochDay());
+ dateDayVector.setSafe(1, (int) LocalDate.now().toEpochDay() + 1);
+ dateDayVector.setSafe(2, (int) LocalDate.now().toEpochDay() + 2);
+
+ dateMillisVector.setSafe(0, LocalDate.now().toEpochDay() * 86400000L);
+ dateMillisVector.setSafe(1, (LocalDate.now().toEpochDay() + 1) * 86400000L);
+ dateMillisVector.setSafe(2, (LocalDate.now().toEpochDay() + 2) * 86400000L);
+
+ File dataFile = new File(TMP, "testWriteDates.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(dateDayVector.get(row), record.get("dateDay"));
+ assertEquals(
+ dateMillisVector.get(row), ((long) (Integer) record.get("dateMillis")) * 86400000L);
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableDates() throws Exception {
+
+ // Field definitions
+ FieldType dateDayField = new FieldType(true, new ArrowType.Date(DateUnit.DAY), null);
+ FieldType dateMillisField = new FieldType(true, new ArrowType.Date(DateUnit.MILLISECOND), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ DateDayVector dateDayVector =
+ new DateDayVector(new Field("dateDay", dateDayField, null), allocator);
+ DateMilliVector dateMillisVector =
+ new DateMilliVector(new Field("dateMillis", dateMillisField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors = Arrays.asList(dateDayVector, dateMillisVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ dateDayVector.setNull(0);
+ dateDayVector.setSafe(1, 0);
+ dateDayVector.setSafe(2, (int) LocalDate.now().toEpochDay());
+
+ dateMillisVector.setNull(0);
+ dateMillisVector.setSafe(1, 0);
+ dateMillisVector.setSafe(2, LocalDate.now().toEpochDay() * 86400000L);
+
+ File dataFile = new File(TMP, "testWriteNullableDates.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("dateDay"));
+ assertNull(record.get("dateMillis"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(dateDayVector.get(row), record.get("dateDay"));
+ assertEquals(
+ dateMillisVector.get(row), ((long) (Integer) record.get("dateMillis")) * 86400000L);
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteTimes() throws Exception {
+
+ // Field definitions
+ FieldType timeSecField = new FieldType(false, new ArrowType.Time(TimeUnit.SECOND, 32), null);
+ FieldType timeMillisField =
+ new FieldType(false, new ArrowType.Time(TimeUnit.MILLISECOND, 32), null);
+ FieldType timeMicrosField =
+ new FieldType(false, new ArrowType.Time(TimeUnit.MICROSECOND, 64), null);
+ FieldType timeNanosField =
+ new FieldType(false, new ArrowType.Time(TimeUnit.NANOSECOND, 64), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeSecVector timeSecVector =
+ new TimeSecVector(new Field("timeSec", timeSecField, null), allocator);
+ TimeMilliVector timeMillisVector =
+ new TimeMilliVector(new Field("timeMillis", timeMillisField, null), allocator);
+ TimeMicroVector timeMicrosVector =
+ new TimeMicroVector(new Field("timeMicros", timeMicrosField, null), allocator);
+ TimeNanoVector timeNanosVector =
+ new TimeNanoVector(new Field("timeNanos", timeNanosField, null), allocator);
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(timeSecVector, timeMillisVector, timeMicrosVector, timeNanosVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timeSecVector.setSafe(0, ZonedDateTime.now().toLocalTime().toSecondOfDay());
+ timeSecVector.setSafe(1, ZonedDateTime.now().toLocalTime().toSecondOfDay() - 1);
+ timeSecVector.setSafe(2, ZonedDateTime.now().toLocalTime().toSecondOfDay() - 2);
+
+ timeMillisVector.setSafe(
+ 0, (int) (ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000000));
+ timeMillisVector.setSafe(
+ 1, (int) (ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000000) - 1000);
+ timeMillisVector.setSafe(
+ 2, (int) (ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000000) - 2000);
+
+ timeMicrosVector.setSafe(0, ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000);
+ timeMicrosVector.setSafe(1, ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000 - 1000000);
+ timeMicrosVector.setSafe(2, ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000 - 2000000);
+
+ timeNanosVector.setSafe(0, ZonedDateTime.now().toLocalTime().toNanoOfDay());
+ timeNanosVector.setSafe(1, ZonedDateTime.now().toLocalTime().toNanoOfDay() - 1000000000);
+ timeNanosVector.setSafe(2, ZonedDateTime.now().toLocalTime().toNanoOfDay() - 2000000000);
+
+ File dataFile = new File(TMP, "testWriteTimes.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(timeSecVector.get(row), (int) (record.get("timeSec")) / 1000);
+ assertEquals(timeMillisVector.get(row), record.get("timeMillis"));
+ assertEquals(timeMicrosVector.get(row), record.get("timeMicros"));
+ // Avro doesn't have time-nanos (mar 2025), so expect column to be saved as micros
+ long nanosAsMicros = (timeNanosVector.get(row) / 1000);
+ assertEquals(nanosAsMicros, (long) record.get("timeNanos"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableTimes() throws Exception {
+
+ // Field definitions
+ FieldType timeSecField = new FieldType(true, new ArrowType.Time(TimeUnit.SECOND, 32), null);
+ FieldType timeMillisField =
+ new FieldType(true, new ArrowType.Time(TimeUnit.MILLISECOND, 32), null);
+ FieldType timeMicrosField =
+ new FieldType(true, new ArrowType.Time(TimeUnit.MICROSECOND, 64), null);
+ FieldType timeNanosField =
+ new FieldType(true, new ArrowType.Time(TimeUnit.NANOSECOND, 64), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeSecVector timeSecVector =
+ new TimeSecVector(new Field("timeSec", timeSecField, null), allocator);
+ TimeMilliVector timeMillisVector =
+ new TimeMilliVector(new Field("timeMillis", timeMillisField, null), allocator);
+ TimeMicroVector timeMicrosVector =
+ new TimeMicroVector(new Field("timeMicros", timeMicrosField, null), allocator);
+ TimeNanoVector timeNanosVector =
+ new TimeNanoVector(new Field("timeNanos", timeNanosField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(timeSecVector, timeMillisVector, timeMicrosVector, timeNanosVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timeSecVector.setNull(0);
+ timeSecVector.setSafe(1, 0);
+ timeSecVector.setSafe(2, ZonedDateTime.now().toLocalTime().toSecondOfDay());
+
+ timeMillisVector.setNull(0);
+ timeMillisVector.setSafe(1, 0);
+ timeMillisVector.setSafe(
+ 2, (int) (ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000000));
+
+ timeMicrosVector.setNull(0);
+ timeMicrosVector.setSafe(1, 0);
+ timeMicrosVector.setSafe(2, ZonedDateTime.now().toLocalTime().toNanoOfDay() / 1000);
+
+ timeNanosVector.setNull(0);
+ timeNanosVector.setSafe(1, 0);
+ timeNanosVector.setSafe(2, ZonedDateTime.now().toLocalTime().toNanoOfDay());
+
+ File dataFile = new File(TMP, "testWriteNullableTimes.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("timeSec"));
+ assertNull(record.get("timeMillis"));
+ assertNull(record.get("timeMicros"));
+ assertNull(record.get("timeNanos"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(timeSecVector.get(row), ((int) record.get("timeSec") / 1000));
+ assertEquals(timeMillisVector.get(row), record.get("timeMillis"));
+ assertEquals(timeMicrosVector.get(row), record.get("timeMicros"));
+ // Avro doesn't have time-nanos (mar 2025), so expect column to be saved as micros
+ long nanosAsMicros = (timeNanosVector.get(row) / 1000);
+ assertEquals(nanosAsMicros, (long) record.get("timeNanos"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteZoneAwareTimestamps() throws Exception {
+
+ // Field definitions
+ FieldType timestampSecField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.SECOND, "UTC"), null);
+ FieldType timestampMillisField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC"), null);
+ FieldType timestampMicrosField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC"), null);
+ FieldType timestampNanosField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC"), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeStampSecTZVector timestampSecVector =
+ new TimeStampSecTZVector(new Field("timestampSec", timestampSecField, null), allocator);
+ TimeStampMilliTZVector timestampMillisVector =
+ new TimeStampMilliTZVector(
+ new Field("timestampMillis", timestampMillisField, null), allocator);
+ TimeStampMicroTZVector timestampMicrosVector =
+ new TimeStampMicroTZVector(
+ new Field("timestampMicros", timestampMicrosField, null), allocator);
+ TimeStampNanoTZVector timestampNanosVector =
+ new TimeStampNanoTZVector(
+ new Field("timestampNanos", timestampNanosField, null), allocator);
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ timestampSecVector, timestampMillisVector, timestampMicrosVector, timestampNanosVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timestampSecVector.setSafe(0, (int) Instant.now().getEpochSecond());
+ timestampSecVector.setSafe(1, (int) Instant.now().getEpochSecond() - 1);
+ timestampSecVector.setSafe(2, (int) Instant.now().getEpochSecond() - 2);
+
+ timestampMillisVector.setSafe(0, (int) Instant.now().toEpochMilli());
+ timestampMillisVector.setSafe(1, (int) Instant.now().toEpochMilli() - 1000);
+ timestampMillisVector.setSafe(2, (int) Instant.now().toEpochMilli() - 2000);
+
+ timestampMicrosVector.setSafe(0, Instant.now().toEpochMilli() * 1000);
+ timestampMicrosVector.setSafe(1, (Instant.now().toEpochMilli() - 1000) * 1000);
+ timestampMicrosVector.setSafe(2, (Instant.now().toEpochMilli() - 2000) * 1000);
+
+ timestampNanosVector.setSafe(0, Instant.now().toEpochMilli() * 1000000);
+ timestampNanosVector.setSafe(1, (Instant.now().toEpochMilli() - 1000) * 1000000);
+ timestampNanosVector.setSafe(2, (Instant.now().toEpochMilli() - 2000) * 1000000);
+
+ File dataFile = new File(TMP, "testWriteZoneAwareTimestamps.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ timestampSecVector.get(row), (int) ((long) record.get("timestampSec") / 1000));
+ assertEquals(timestampMillisVector.get(row), (int) (long) record.get("timestampMillis"));
+ assertEquals(timestampMicrosVector.get(row), record.get("timestampMicros"));
+ assertEquals(timestampNanosVector.get(row), record.get("timestampNanos"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableZoneAwareTimestamps() throws Exception {
+
+ // Field definitions
+ FieldType timestampSecField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.SECOND, "UTC"), null);
+ FieldType timestampMillisField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC"), null);
+ FieldType timestampMicrosField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC"), null);
+ FieldType timestampNanosField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC"), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeStampSecTZVector timestampSecVector =
+ new TimeStampSecTZVector(new Field("timestampSec", timestampSecField, null), allocator);
+ TimeStampMilliTZVector timestampMillisVector =
+ new TimeStampMilliTZVector(
+ new Field("timestampMillis", timestampMillisField, null), allocator);
+ TimeStampMicroTZVector timestampMicrosVector =
+ new TimeStampMicroTZVector(
+ new Field("timestampMicros", timestampMicrosField, null), allocator);
+ TimeStampNanoTZVector timestampNanosVector =
+ new TimeStampNanoTZVector(
+ new Field("timestampNanos", timestampNanosField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ timestampSecVector, timestampMillisVector, timestampMicrosVector, timestampNanosVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timestampSecVector.setNull(0);
+ timestampSecVector.setSafe(1, 0);
+ timestampSecVector.setSafe(2, (int) Instant.now().getEpochSecond());
+
+ timestampMillisVector.setNull(0);
+ timestampMillisVector.setSafe(1, 0);
+ timestampMillisVector.setSafe(2, (int) Instant.now().toEpochMilli());
+
+ timestampMicrosVector.setNull(0);
+ timestampMicrosVector.setSafe(1, 0);
+ timestampMicrosVector.setSafe(2, Instant.now().toEpochMilli() * 1000);
+
+ timestampNanosVector.setNull(0);
+ timestampNanosVector.setSafe(1, 0);
+ timestampNanosVector.setSafe(2, Instant.now().toEpochMilli() * 1000000);
+
+ File dataFile = new File(TMP, "testWriteNullableZoneAwareTimestamps.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("timestampSec"));
+ assertNull(record.get("timestampMillis"));
+ assertNull(record.get("timestampMicros"));
+ assertNull(record.get("timestampNanos"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ timestampSecVector.get(row), (int) ((long) record.get("timestampSec") / 1000));
+ assertEquals(timestampMillisVector.get(row), (int) (long) record.get("timestampMillis"));
+ assertEquals(timestampMicrosVector.get(row), record.get("timestampMicros"));
+ assertEquals(timestampNanosVector.get(row), record.get("timestampNanos"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteLocalTimestamps() throws Exception {
+
+ // Field definitions
+ FieldType timestampSecField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.SECOND, null), null);
+ FieldType timestampMillisField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), null);
+ FieldType timestampMicrosField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), null);
+ FieldType timestampNanosField =
+ new FieldType(false, new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeStampSecVector timestampSecVector =
+ new TimeStampSecVector(new Field("timestampSec", timestampSecField, null), allocator);
+ TimeStampMilliVector timestampMillisVector =
+ new TimeStampMilliVector(
+ new Field("timestampMillis", timestampMillisField, null), allocator);
+ TimeStampMicroVector timestampMicrosVector =
+ new TimeStampMicroVector(
+ new Field("timestampMicros", timestampMicrosField, null), allocator);
+ TimeStampNanoVector timestampNanosVector =
+ new TimeStampNanoVector(new Field("timestampNanos", timestampNanosField, null), allocator);
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ timestampSecVector, timestampMillisVector, timestampMicrosVector, timestampNanosVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timestampSecVector.setSafe(0, (int) Instant.now().getEpochSecond());
+ timestampSecVector.setSafe(1, (int) Instant.now().getEpochSecond() - 1);
+ timestampSecVector.setSafe(2, (int) Instant.now().getEpochSecond() - 2);
+
+ timestampMillisVector.setSafe(0, (int) Instant.now().toEpochMilli());
+ timestampMillisVector.setSafe(1, (int) Instant.now().toEpochMilli() - 1000);
+ timestampMillisVector.setSafe(2, (int) Instant.now().toEpochMilli() - 2000);
+
+ timestampMicrosVector.setSafe(0, Instant.now().toEpochMilli() * 1000);
+ timestampMicrosVector.setSafe(1, (Instant.now().toEpochMilli() - 1000) * 1000);
+ timestampMicrosVector.setSafe(2, (Instant.now().toEpochMilli() - 2000) * 1000);
+
+ timestampNanosVector.setSafe(0, Instant.now().toEpochMilli() * 1000000);
+ timestampNanosVector.setSafe(1, (Instant.now().toEpochMilli() - 1000) * 1000000);
+ timestampNanosVector.setSafe(2, (Instant.now().toEpochMilli() - 2000) * 1000000);
+
+ File dataFile = new File(TMP, "testWriteZoneAwareTimestamps.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ timestampSecVector.get(row), (int) ((long) record.get("timestampSec") / 1000));
+ assertEquals(timestampMillisVector.get(row), (int) (long) record.get("timestampMillis"));
+ assertEquals(timestampMicrosVector.get(row), record.get("timestampMicros"));
+ assertEquals(timestampNanosVector.get(row), record.get("timestampNanos"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableLocalTimestamps() throws Exception {
+
+ // Field definitions
+ FieldType timestampSecField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.SECOND, null), null);
+ FieldType timestampMillisField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.MILLISECOND, null), null);
+ FieldType timestampMicrosField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.MICROSECOND, null), null);
+ FieldType timestampNanosField =
+ new FieldType(true, new ArrowType.Timestamp(TimeUnit.NANOSECOND, null), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ TimeStampSecVector timestampSecVector =
+ new TimeStampSecVector(new Field("timestampSec", timestampSecField, null), allocator);
+ TimeStampMilliVector timestampMillisVector =
+ new TimeStampMilliVector(
+ new Field("timestampMillis", timestampMillisField, null), allocator);
+ TimeStampMicroVector timestampMicrosVector =
+ new TimeStampMicroVector(
+ new Field("timestampMicros", timestampMicrosField, null), allocator);
+ TimeStampNanoVector timestampNanosVector =
+ new TimeStampNanoVector(new Field("timestampNanos", timestampNanosField, null), allocator);
+
+ int rowCount = 3;
+
+ // Set up VSR
+ List vectors =
+ Arrays.asList(
+ timestampSecVector, timestampMillisVector, timestampMicrosVector, timestampNanosVector);
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ timestampSecVector.setNull(0);
+ timestampSecVector.setSafe(1, 0);
+ timestampSecVector.setSafe(2, (int) Instant.now().getEpochSecond());
+
+ timestampMillisVector.setNull(0);
+ timestampMillisVector.setSafe(1, 0);
+ timestampMillisVector.setSafe(2, (int) Instant.now().toEpochMilli());
+
+ timestampMicrosVector.setNull(0);
+ timestampMicrosVector.setSafe(1, 0);
+ timestampMicrosVector.setSafe(2, Instant.now().toEpochMilli() * 1000);
+
+ timestampNanosVector.setNull(0);
+ timestampNanosVector.setSafe(1, 0);
+ timestampNanosVector.setSafe(2, Instant.now().toEpochMilli() * 1000000);
+
+ File dataFile = new File(TMP, "testWriteNullableZoneAwareTimestamps.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+
+ // Read and check values
+ GenericRecord record = datumReader.read(null, decoder);
+ assertNull(record.get("timestampSec"));
+ assertNull(record.get("timestampMillis"));
+ assertNull(record.get("timestampMicros"));
+ assertNull(record.get("timestampNanos"));
+
+ for (int row = 1; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(
+ timestampSecVector.get(row), (int) ((long) record.get("timestampSec") / 1000));
+ assertEquals(timestampMillisVector.get(row), (int) (long) record.get("timestampMillis"));
+ assertEquals(timestampMicrosVector.get(row), record.get("timestampMicros"));
+ assertEquals(timestampNanosVector.get(row), record.get("timestampNanos"));
+ }
+ }
+ }
+ }
+
+ // Data production for containers of primitive and logical types, nullable and non-nullable
+
+ @Test
+ public void testWriteLists() throws Exception {
+
+ // Field definitions
+ FieldType intListField = new FieldType(false, new ArrowType.List(), null);
+ FieldType stringListField = new FieldType(false, new ArrowType.List(), null);
+ FieldType dateListField = new FieldType(false, new ArrowType.List(), null);
+
+ Field intField = new Field("item", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field stringField = new Field("item", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field dateField =
+ new Field("item", FieldType.notNullable(new ArrowType.Date(DateUnit.DAY)), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ ListVector intListVector = new ListVector("intList", allocator, intListField, null);
+ ListVector stringListVector = new ListVector("stringList", allocator, stringListField, null);
+ ListVector dateListVector = new ListVector("dateList", allocator, dateListField, null);
+
+ intListVector.initializeChildrenFromFields(Arrays.asList(intField));
+ stringListVector.initializeChildrenFromFields(Arrays.asList(stringField));
+ dateListVector.initializeChildrenFromFields(Arrays.asList(dateField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(intListVector, stringListVector, dateListVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ FieldWriter intListWriter = intListVector.getWriter();
+ FieldWriter stringListWriter = stringListVector.getWriter();
+ FieldWriter dateListWriter = dateListVector.getWriter();
+
+ // Set test data for intList
+ for (int i = 0; i < rowCount; i++) {
+ intListWriter.startList();
+ for (int j = 0; j < 5 - i; j++) {
+ intListWriter.writeInt(j);
+ }
+ intListWriter.endList();
+ }
+
+ // Set test data for stringList
+ for (int i = 0; i < rowCount; i++) {
+ stringListWriter.startList();
+ for (int j = 0; j < 5 - i; j++) {
+ stringListWriter.writeVarChar("string" + j);
+ }
+ stringListWriter.endList();
+ }
+
+ // Set test data for dateList
+ for (int i = 0; i < rowCount; i++) {
+ dateListWriter.startList();
+ for (int j = 0; j < 5 - i; j++) {
+ dateListWriter.writeDateDay((int) LocalDate.now().plusDays(j).toEpochDay());
+ }
+ dateListWriter.endList();
+ }
+
+ // Update count for the vectors
+ intListVector.setValueCount(rowCount);
+ stringListVector.setValueCount(rowCount);
+ dateListVector.setValueCount(rowCount);
+
+ File dataFile = new File(TMP, "testWriteLists.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(intListVector.getObject(row), record.get("intList"));
+ assertEquals(dateListVector.getObject(row), record.get("dateList"));
+ // Handle conversion from Arrow Text type
+ List> vectorList = stringListVector.getObject(row);
+ List> recordList = (List>) record.get("stringList");
+ assertEquals(vectorList.size(), recordList.size());
+ for (int i = 0; i < vectorList.size(); i++) {
+ assertEquals(vectorList.get(i).toString(), recordList.get(i).toString());
+ }
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableLists() throws Exception {
+
+ // Field definitions
+ FieldType nullListType = new FieldType(true, new ArrowType.List(), null);
+ FieldType nonNullListType = new FieldType(false, new ArrowType.List(), null);
+
+ Field nullFieldType = new Field("item", FieldType.nullable(new ArrowType.Int(32, true)), null);
+ Field nonNullFieldType =
+ new Field("item", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ ListVector nullEntriesVector =
+ new ListVector("nullEntriesVector", allocator, nonNullListType, null);
+ ListVector nullListVector = new ListVector("nullListVector", allocator, nullListType, null);
+ ListVector nullBothVector = new ListVector("nullBothVector", allocator, nullListType, null);
+
+ nullEntriesVector.initializeChildrenFromFields(Arrays.asList(nullFieldType));
+ nullListVector.initializeChildrenFromFields(Arrays.asList(nonNullFieldType));
+ nullBothVector.initializeChildrenFromFields(Arrays.asList(nullFieldType));
+
+ // Set up VSR
+ List vectors = Arrays.asList(nullEntriesVector, nullListVector, nullBothVector);
+ int rowCount = 4;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data for nullEntriesVector
+ FieldWriter nullEntriesWriter = nullEntriesVector.getWriter();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeNull();
+ nullEntriesWriter.integer().writeNull();
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(0);
+ nullEntriesWriter.integer().writeInt(0);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(123);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.endList();
+
+ // Set test data for nullListVector
+ FieldWriter nullListWriter = nullListVector.getWriter();
+ nullListWriter.writeNull();
+ nullListWriter.setPosition(1); // writeNull() does not inc. idx() on list vector
+ nullListWriter.startList();
+ nullListWriter.integer().writeInt(0);
+ nullListWriter.integer().writeInt(0);
+ nullListWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(123);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.endList();
+
+ // Set test data for nullBothVector
+ FieldWriter nullBothWriter = nullBothVector.getWriter();
+ nullBothWriter.writeNull();
+ nullBothWriter.setPosition(1);
+ nullBothWriter.startList();
+ nullBothWriter.integer().writeNull();
+ nullBothWriter.integer().writeNull();
+ nullBothWriter.endList();
+ nullListWriter.startList();
+ nullListWriter.integer().writeInt(0);
+ nullListWriter.integer().writeInt(0);
+ nullListWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(123);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+
+ // Update count for the vectors
+ nullListVector.setValueCount(4);
+ nullEntriesVector.setValueCount(4);
+ nullBothVector.setValueCount(4);
+
+ File dataFile = new File(TMP, "testWriteNullableLists.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ for (String list :
+ Arrays.asList("nullEntriesVector", "nullListVector", "nullBothVector")) {
+ ListVector vector = (ListVector) root.getVector(list);
+ Object recordField = record.get(list);
+ if (vector.isNull(row)) {
+ assertNull(recordField);
+ } else {
+ assertEquals(vector.getObject(row), recordField);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteFixedLists() throws Exception {
+
+ // Field definitions
+ FieldType intListField = new FieldType(false, new ArrowType.FixedSizeList(5), null);
+ FieldType stringListField = new FieldType(false, new ArrowType.FixedSizeList(5), null);
+ FieldType dateListField = new FieldType(false, new ArrowType.FixedSizeList(5), null);
+
+ Field intField = new Field("item", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field stringField = new Field("item", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field dateField =
+ new Field("item", FieldType.notNullable(new ArrowType.Date(DateUnit.DAY)), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ FixedSizeListVector intListVector =
+ new FixedSizeListVector("intList", allocator, intListField, null);
+ FixedSizeListVector stringListVector =
+ new FixedSizeListVector("stringList", allocator, stringListField, null);
+ FixedSizeListVector dateListVector =
+ new FixedSizeListVector("dateList", allocator, dateListField, null);
+
+ intListVector.initializeChildrenFromFields(Arrays.asList(intField));
+ stringListVector.initializeChildrenFromFields(Arrays.asList(stringField));
+ dateListVector.initializeChildrenFromFields(Arrays.asList(dateField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(intListVector, stringListVector, dateListVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ FieldWriter intListWriter = intListVector.getWriter();
+ FieldWriter stringListWriter = stringListVector.getWriter();
+ FieldWriter dateListWriter = dateListVector.getWriter();
+
+ // Set test data for intList
+ for (int i = 0; i < rowCount; i++) {
+ intListWriter.startList();
+ for (int j = 0; j < 5; j++) {
+ intListWriter.writeInt(j);
+ }
+ intListWriter.endList();
+ }
+
+ // Set test data for stringList
+ for (int i = 0; i < rowCount; i++) {
+ stringListWriter.startList();
+ for (int j = 0; j < 5; j++) {
+ stringListWriter.writeVarChar("string" + j);
+ }
+ stringListWriter.endList();
+ }
+
+ // Set test data for dateList
+ for (int i = 0; i < rowCount; i++) {
+ dateListWriter.startList();
+ for (int j = 0; j < 5; j++) {
+ dateListWriter.writeDateDay((int) LocalDate.now().plusDays(j).toEpochDay());
+ }
+ dateListWriter.endList();
+ }
+ File dataFile = new File(TMP, "testWriteFixedLists.avro");
+
+ // Update count for the vectors
+ intListVector.setValueCount(rowCount);
+ stringListVector.setValueCount(rowCount);
+ dateListVector.setValueCount(rowCount);
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertEquals(intListVector.getObject(row), record.get("intList"));
+ assertEquals(dateListVector.getObject(row), record.get("dateList"));
+ // Handle conversion from Arrow Text type
+ List> vectorList = stringListVector.getObject(row);
+ List> recordList = (List>) record.get("stringList");
+ assertEquals(vectorList.size(), recordList.size());
+ for (int i = 0; i < vectorList.size(); i++) {
+ assertEquals(vectorList.get(i).toString(), recordList.get(i).toString());
+ }
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableFixedLists() throws Exception {
+
+ // Field definitions
+ FieldType nullListType = new FieldType(true, new ArrowType.FixedSizeList(2), null);
+ FieldType nonNullListType = new FieldType(false, new ArrowType.FixedSizeList(2), null);
+
+ Field nullFieldType = new Field("item", FieldType.nullable(new ArrowType.Int(32, true)), null);
+ Field nonNullFieldType =
+ new Field("item", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ FixedSizeListVector nullEntriesVector =
+ new FixedSizeListVector("nullEntriesVector", allocator, nonNullListType, null);
+ FixedSizeListVector nullListVector =
+ new FixedSizeListVector("nullListVector", allocator, nullListType, null);
+ FixedSizeListVector nullBothVector =
+ new FixedSizeListVector("nullBothVector", allocator, nullListType, null);
+
+ nullEntriesVector.initializeChildrenFromFields(Arrays.asList(nullFieldType));
+ nullListVector.initializeChildrenFromFields(Arrays.asList(nonNullFieldType));
+ nullBothVector.initializeChildrenFromFields(Arrays.asList(nullFieldType));
+
+ // Set up VSR
+ List vectors = Arrays.asList(nullEntriesVector, nullListVector, nullBothVector);
+ int rowCount = 4;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data for nullEntriesVector
+ FieldWriter nullEntriesWriter = nullEntriesVector.getWriter();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeNull();
+ nullEntriesWriter.integer().writeNull();
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(0);
+ nullEntriesWriter.integer().writeInt(0);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(123);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.endList();
+
+ // Set test data for nullListVector
+ FieldWriter nullListWriter = nullListVector.getWriter();
+ nullListWriter.writeNull();
+ nullListWriter.setPosition(1); // writeNull() does not inc. idx() on list vector
+ nullListWriter.startList();
+ nullListWriter.integer().writeInt(123);
+ nullListWriter.integer().writeInt(456);
+ nullListWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(12345);
+ nullEntriesWriter.integer().writeInt(67891);
+ nullEntriesWriter.endList();
+
+ // Set test data for nullBothVector
+ FieldWriter nullBothWriter = nullBothVector.getWriter();
+ nullBothWriter.writeNull();
+ nullBothWriter.setPosition(1);
+ nullBothWriter.startList();
+ nullListWriter.integer().writeNull();
+ nullListWriter.integer().writeNull();
+ nullBothWriter.endList();
+ nullListWriter.startList();
+ nullListWriter.integer().writeInt(123);
+ nullListWriter.integer().writeInt(456);
+ nullListWriter.endList();
+ nullEntriesWriter.startList();
+ nullEntriesWriter.integer().writeInt(789);
+ nullEntriesWriter.integer().writeInt(456);
+ nullEntriesWriter.endList();
+
+ // Update count for the vectors
+ nullListVector.setValueCount(4);
+ nullEntriesVector.setValueCount(4);
+ nullBothVector.setValueCount(4);
+
+ File dataFile = new File(TMP, "testWriteNullableFixedLists.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ for (String list :
+ Arrays.asList("nullEntriesVector", "nullListVector", "nullBothVector")) {
+ FixedSizeListVector vector = (FixedSizeListVector) root.getVector(list);
+ Object recordField = record.get(list);
+ if (vector.isNull(row)) {
+ assertNull(recordField);
+ } else {
+ assertEquals(vector.getObject(row), recordField);
+ }
+ }
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteMap() throws Exception {
+
+ // Field definitions
+ FieldType intMapField = new FieldType(false, new ArrowType.Map(false), null);
+ FieldType stringMapField = new FieldType(false, new ArrowType.Map(false), null);
+ FieldType dateMapField = new FieldType(false, new ArrowType.Map(false), null);
+
+ Field keyField = new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field intField = new Field("value", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field stringField = new Field("value", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field dateField =
+ new Field("value", FieldType.notNullable(new ArrowType.Date(DateUnit.DAY)), null);
+
+ Field intEntryField =
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(keyField, intField));
+ Field stringEntryField =
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(keyField, stringField));
+ Field dateEntryField =
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(keyField, dateField));
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ MapVector intMapVector = new MapVector("intMap", allocator, intMapField, null);
+ MapVector stringMapVector = new MapVector("stringMap", allocator, stringMapField, null);
+ MapVector dateMapVector = new MapVector("dateMap", allocator, dateMapField, null);
+
+ intMapVector.initializeChildrenFromFields(Arrays.asList(intEntryField));
+ stringMapVector.initializeChildrenFromFields(Arrays.asList(stringEntryField));
+ dateMapVector.initializeChildrenFromFields(Arrays.asList(dateEntryField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(intMapVector, stringMapVector, dateMapVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Total number of entries that will be writen to each vector
+ int entryCount = 5 + 4 + 3;
+
+ // Set test data for intList
+ BaseWriter.MapWriter writer = intMapVector.getWriter();
+ for (int i = 0; i < rowCount; i++) {
+ writer.startMap();
+ for (int j = 0; j < 5 - i; j++) {
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key" + j);
+ writer.value().integer().writeInt(j);
+ writer.endEntry();
+ }
+ writer.endMap();
+ }
+
+ // Update count for data vector (map writer does not do this)
+ intMapVector.getDataVector().setValueCount(entryCount);
+
+ // Set test data for stringList
+ BaseWriter.MapWriter stringWriter = stringMapVector.getWriter();
+ for (int i = 0; i < rowCount; i++) {
+ stringWriter.startMap();
+ for (int j = 0; j < 5 - i; j++) {
+ stringWriter.startEntry();
+ stringWriter.key().varChar().writeVarChar("key" + j);
+ stringWriter.value().varChar().writeVarChar("string" + j);
+ stringWriter.endEntry();
+ }
+ stringWriter.endMap();
+ }
+
+ // Update count for the vectors
+ intMapVector.setValueCount(rowCount);
+ stringMapVector.setValueCount(rowCount);
+ dateMapVector.setValueCount(rowCount);
+
+ // Update count for data vector (map writer does not do this)
+ stringMapVector.getDataVector().setValueCount(entryCount);
+
+ // Set test data for dateList
+ BaseWriter.MapWriter dateWriter = dateMapVector.getWriter();
+ for (int i = 0; i < rowCount; i++) {
+ dateWriter.startMap();
+ for (int j = 0; j < 5 - i; j++) {
+ dateWriter.startEntry();
+ dateWriter.key().varChar().writeVarChar("key" + j);
+ dateWriter.value().dateDay().writeDateDay((int) LocalDate.now().plusDays(j).toEpochDay());
+ dateWriter.endEntry();
+ }
+ dateWriter.endMap();
+ }
+
+ // Update count for data vector (map writer does not do this)
+ dateMapVector.getDataVector().setValueCount(entryCount);
+
+ File dataFile = new File(TMP, "testWriteMap.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ Map intMap = convertMap(intMapVector.getObject(row));
+ Map stringMap = convertMap(stringMapVector.getObject(row));
+ Map dateMap = convertMap(dateMapVector.getObject(row));
+ compareMaps(intMap, (Map) record.get("intMap"));
+ compareMaps(stringMap, (Map) record.get("stringMap"));
+ compareMaps(dateMap, (Map) record.get("dateMap"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableMap() throws Exception {
+
+ // Field definitions
+ FieldType nullMapType = new FieldType(true, new ArrowType.Map(false), null);
+ FieldType nonNullMapType = new FieldType(false, new ArrowType.Map(false), null);
+
+ Field keyField = new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field nullFieldType = new Field("value", FieldType.nullable(new ArrowType.Int(32, true)), null);
+ Field nonNullFieldType =
+ new Field("value", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field nullEntryField =
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(keyField, nullFieldType));
+ Field nonNullEntryField =
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(keyField, nonNullFieldType));
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ MapVector nullEntriesVector =
+ new MapVector("nullEntriesVector", allocator, nonNullMapType, null);
+ MapVector nullMapVector = new MapVector("nullMapVector", allocator, nullMapType, null);
+ MapVector nullBothVector = new MapVector("nullBothVector", allocator, nullMapType, null);
+
+ nullEntriesVector.initializeChildrenFromFields(Arrays.asList(nullEntryField));
+ nullMapVector.initializeChildrenFromFields(Arrays.asList(nonNullEntryField));
+ nullBothVector.initializeChildrenFromFields(Arrays.asList(nullEntryField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(nullEntriesVector, nullMapVector, nullBothVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data for intList
+ BaseWriter.MapWriter writer = nullEntriesVector.getWriter();
+ writer.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key0");
+ writer.value().integer().writeNull();
+ writer.endEntry();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key1");
+ writer.value().integer().writeNull();
+ writer.endEntry();
+ writer.endMap();
+ writer.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key2");
+ writer.value().integer().writeInt(0);
+ writer.endEntry();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key3");
+ writer.value().integer().writeInt(0);
+ writer.endEntry();
+ writer.endMap();
+ writer.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key4");
+ writer.value().integer().writeInt(123);
+ writer.endEntry();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key5");
+ writer.value().integer().writeInt(456);
+ writer.endEntry();
+ writer.endMap();
+
+ // Set test data for stringList
+ BaseWriter.MapWriter nullMapWriter = nullMapVector.getWriter();
+ nullMapWriter.writeNull();
+ nullMapWriter.setPosition(1); // writeNull() does not inc. idx() on map (list) vector
+ nullMapWriter.startMap();
+ nullMapWriter.startEntry();
+ nullMapWriter.key().varChar().writeVarChar("key2");
+ nullMapWriter.value().integer().writeInt(0);
+ nullMapWriter.endEntry();
+ writer.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key3");
+ writer.value().integer().writeInt(0);
+ writer.endEntry();
+ nullMapWriter.endMap();
+ nullMapWriter.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key4");
+ writer.value().integer().writeInt(123);
+ writer.endEntry();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key5");
+ writer.value().integer().writeInt(456);
+ writer.endEntry();
+ nullMapWriter.endMap();
+
+ // Set test data for dateList
+ BaseWriter.MapWriter nullBothWriter = nullBothVector.getWriter();
+ nullBothWriter.writeNull();
+ nullBothWriter.setPosition(1);
+ nullBothWriter.startMap();
+ nullBothWriter.startEntry();
+ nullBothWriter.key().varChar().writeVarChar("key2");
+ nullBothWriter.value().integer().writeNull();
+ nullBothWriter.endEntry();
+ nullBothWriter.startEntry();
+ nullBothWriter.key().varChar().writeVarChar("key3");
+ nullBothWriter.value().integer().writeNull();
+ nullBothWriter.endEntry();
+ nullBothWriter.endMap();
+ nullBothWriter.startMap();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key4");
+ writer.value().integer().writeInt(123);
+ writer.endEntry();
+ writer.startEntry();
+ writer.key().varChar().writeVarChar("key5");
+ writer.value().integer().writeInt(456);
+ writer.endEntry();
+ nullBothWriter.endMap();
+
+ // Update count for the vectors
+ nullEntriesVector.setValueCount(3);
+ nullMapVector.setValueCount(3);
+ nullBothVector.setValueCount(3);
+
+ File dataFile = new File(TMP, "testWriteNullableMap.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ Map intMap = convertMap(nullEntriesVector.getObject(row));
+ Map stringMap = convertMap(nullMapVector.getObject(row));
+ Map dateMap = convertMap(nullBothVector.getObject(row));
+ compareMaps(intMap, (Map) record.get("nullEntriesVector"));
+ compareMaps(stringMap, (Map) record.get("nullMapVector"));
+ compareMaps(dateMap, (Map) record.get("nullBothVector"));
+ }
+ }
+ }
+ }
+
+ private Map convertMap(List> entryList) {
+
+ if (entryList == null) {
+ return null;
+ }
+
+ Map map = new HashMap<>();
+ JsonStringArrayList> structList = (JsonStringArrayList>) entryList;
+ for (Object entry : structList) {
+ JsonStringHashMap structEntry = (JsonStringHashMap) entry;
+ String key = structEntry.get(MapVector.KEY_NAME).toString();
+ Object value = structEntry.get(MapVector.VALUE_NAME);
+ map.put(key, value);
+ }
+ return map;
+ }
+
+ private void compareMaps(Map expected, Map, ?> actual) {
+ if (expected == null) {
+ assertNull(actual);
+ } else {
+ assertEquals(expected.size(), actual.size());
+ for (Object key : actual.keySet()) {
+ assertTrue(expected.containsKey(key.toString()));
+ Object actualValue = actual.get(key);
+ if (actualValue instanceof Utf8) {
+ assertEquals(expected.get(key.toString()).toString(), actualValue.toString());
+ } else {
+ assertEquals(expected.get(key.toString()), actual.get(key));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteStruct() throws Exception {
+
+ // Field definitions
+ FieldType structFieldType = new FieldType(false, new ArrowType.Struct(), null);
+ Field intField =
+ new Field("intField", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field stringField = new Field("stringField", FieldType.notNullable(new ArrowType.Utf8()), null);
+ Field dateField =
+ new Field("dateField", FieldType.notNullable(new ArrowType.Date(DateUnit.DAY)), null);
+
+ // Create empty vector
+ BufferAllocator allocator = new RootAllocator();
+ StructVector structVector = new StructVector("struct", allocator, structFieldType, null);
+ structVector.initializeChildrenFromFields(Arrays.asList(intField, stringField, dateField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(structVector);
+ int rowCount = 3;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data
+ BaseWriter.StructWriter structWriter = structVector.getWriter();
+
+ for (int i = 0; i < rowCount; i++) {
+ structWriter.start();
+ structWriter.integer("intField").writeInt(i);
+ structWriter.varChar("stringField").writeVarChar("string" + i);
+ structWriter.dateDay("dateField").writeDateDay((int) LocalDate.now().toEpochDay() + i);
+ structWriter.end();
+ }
+
+ File dataFile = new File(TMP, "testWriteStruct.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Update count for the vector
+ structVector.setValueCount(rowCount);
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ assertNotNull(record.get("struct"));
+ GenericRecord structRecord = (GenericRecord) record.get("struct");
+ assertEquals(row, structRecord.get("intField"));
+ assertEquals("string" + row, structRecord.get("stringField").toString());
+ assertEquals((int) LocalDate.now().toEpochDay() + row, structRecord.get("dateField"));
+ }
+ }
+ }
+ }
+
+ @Test
+ public void testWriteNullableStructs() throws Exception {
+
+ // Field definitions
+ FieldType structFieldType = new FieldType(false, new ArrowType.Struct(), null);
+ FieldType nullableStructFieldType = new FieldType(true, new ArrowType.Struct(), null);
+ Field intField =
+ new Field("intField", FieldType.notNullable(new ArrowType.Int(32, true)), null);
+ Field nullableIntField =
+ new Field("nullableIntField", FieldType.nullable(new ArrowType.Int(32, true)), null);
+
+ // Create empty vectors
+ BufferAllocator allocator = new RootAllocator();
+ StructVector structVector = new StructVector("struct", allocator, structFieldType, null);
+ StructVector nullableStructVector =
+ new StructVector("nullableStruct", allocator, nullableStructFieldType, null);
+ structVector.initializeChildrenFromFields(Arrays.asList(intField, nullableIntField));
+ nullableStructVector.initializeChildrenFromFields(Arrays.asList(intField, nullableIntField));
+
+ // Set up VSR
+ List vectors = Arrays.asList(structVector, nullableStructVector);
+ int rowCount = 4;
+
+ try (VectorSchemaRoot root = new VectorSchemaRoot(vectors)) {
+
+ root.setRowCount(rowCount);
+ root.allocateNew();
+
+ // Set test data for structVector
+ BaseWriter.StructWriter structWriter = structVector.getWriter();
+ for (int i = 0; i < rowCount; i++) {
+ structWriter.setPosition(i);
+ structWriter.start();
+ structWriter.integer("intField").writeInt(i);
+ if (i % 2 == 0) {
+ structWriter.integer("nullableIntField").writeInt(i * 10);
+ } else {
+ structWriter.integer("nullableIntField").writeNull();
+ }
+ structWriter.end();
+ }
+
+ // Set test data for nullableStructVector
+ BaseWriter.StructWriter nullableStructWriter = nullableStructVector.getWriter();
+ for (int i = 0; i < rowCount; i++) {
+ nullableStructWriter.setPosition(i);
+ if (i >= 2) {
+ nullableStructWriter.start();
+ nullableStructWriter.integer("intField").writeInt(i);
+ if (i % 2 == 0) {
+ nullableStructWriter.integer("nullableIntField").writeInt(i * 10);
+ } else {
+ nullableStructWriter.integer("nullableIntField").writeNull();
+ }
+ nullableStructWriter.end();
+ } else {
+ nullableStructWriter.writeNull();
+ }
+ }
+
+ // Update count for the vector
+ structVector.setValueCount(rowCount);
+ nullableStructVector.setValueCount(rowCount);
+
+ File dataFile = new File(TMP, "testWriteNullableStructs.avro");
+
+ // Write an AVRO block using the producer classes
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+ for (int row = 0; row < rowCount; row++) {
+ producer.produce(encoder);
+ }
+ encoder.flush();
+ }
+
+ // Set up reading the AVRO block as a GenericRecord
+ Schema schema = ArrowToAvroUtils.createAvroSchema(root.getSchema().getFields());
+ GenericDatumReader datumReader = new GenericDatumReader<>(schema);
+
+ try (InputStream inputStream = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(inputStream, null);
+ GenericRecord record = null;
+
+ // Read and check values
+ for (int row = 0; row < rowCount; row++) {
+ record = datumReader.read(record, decoder);
+ if (row % 2 == 0) {
+ assertNotNull(record.get("struct"));
+ GenericRecord structRecord = (GenericRecord) record.get("struct");
+ assertEquals(row, structRecord.get("intField"));
+ assertEquals(row * 10, structRecord.get("nullableIntField"));
+ } else {
+ assertNotNull(record.get("struct"));
+ GenericRecord structRecord = (GenericRecord) record.get("struct");
+ assertEquals(row, structRecord.get("intField"));
+ assertNull(structRecord.get("nullableIntField"));
+ }
+ if (row >= 2) {
+ assertNotNull(record.get("nullableStruct"));
+ GenericRecord nullableStructRecord = (GenericRecord) record.get("nullableStruct");
+ assertEquals(row, nullableStructRecord.get("intField"));
+ if (row % 2 == 0) {
+ assertEquals(row * 10, nullableStructRecord.get("nullableIntField"));
+ } else {
+ assertNull(nullableStructRecord.get("nullableIntField"));
+ }
+ } else {
+ assertNull(record.get("nullableStruct"));
+ }
+ }
+ }
+ }
+ }
+}
diff --git a/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroSchemaTest.java b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroSchemaTest.java
new file mode 100644
index 000000000..a05bbc165
--- /dev/null
+++ b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/ArrowToAvroSchemaTest.java
@@ -0,0 +1,1392 @@
+/*
+ * 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 static org.junit.jupiter.api.Assertions.assertEquals;
+
+import java.util.Arrays;
+import java.util.List;
+import org.apache.arrow.vector.types.DateUnit;
+import org.apache.arrow.vector.types.FloatingPointPrecision;
+import org.apache.arrow.vector.types.TimeUnit;
+import org.apache.arrow.vector.types.UnionMode;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
+import org.apache.avro.LogicalTypes;
+import org.apache.avro.Schema;
+import org.junit.jupiter.api.Test;
+
+public class ArrowToAvroSchemaTest {
+
+ // Schema conversion for primitive types, nullable and non-nullable
+
+ @Test
+ public void testConvertNullType() {
+ List fields =
+ Arrays.asList(new Field("nullType", FieldType.notNullable(new ArrowType.Null()), null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(1, schema.getFields().size());
+
+ assertEquals(Schema.Type.NULL, schema.getField("nullType").schema().getType());
+ }
+
+ @Test
+ public void testConvertBooleanTypes() {
+ List fields =
+ Arrays.asList(
+ new Field("nullableBool", FieldType.nullable(new ArrowType.Bool()), null),
+ new Field("nonNullableBool", FieldType.notNullable(new ArrowType.Bool()), null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(2, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableBool").schema().getType());
+ assertEquals(2, schema.getField("nullableBool").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.BOOLEAN, schema.getField("nullableBool").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableBool").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.BOOLEAN, schema.getField("nonNullableBool").schema().getType());
+ }
+
+ @Test
+ public void testConvertIntegralTypes() {
+ List fields =
+ Arrays.asList(
+ new Field("nullableInt8", FieldType.nullable(new ArrowType.Int(8, true)), null),
+ new Field("nonNullableInt8", FieldType.notNullable(new ArrowType.Int(8, true)), null),
+ new Field("nullableUInt8", FieldType.nullable(new ArrowType.Int(8, false)), null),
+ new Field("nonNullableUInt8", FieldType.notNullable(new ArrowType.Int(8, false)), null),
+ new Field("nullableInt16", FieldType.nullable(new ArrowType.Int(16, true)), null),
+ new Field("nonNullableInt16", FieldType.notNullable(new ArrowType.Int(16, true)), null),
+ new Field("nullableUInt16", FieldType.nullable(new ArrowType.Int(16, false)), null),
+ new Field(
+ "nonNullableUInt16", FieldType.notNullable(new ArrowType.Int(16, false)), null),
+ new Field("nullableInt32", FieldType.nullable(new ArrowType.Int(32, true)), null),
+ new Field("nonNullableInt32", FieldType.notNullable(new ArrowType.Int(32, true)), null),
+ new Field("nullableUInt32", FieldType.nullable(new ArrowType.Int(32, false)), null),
+ new Field(
+ "nonNullableUInt32", FieldType.notNullable(new ArrowType.Int(32, false)), null),
+ new Field("nullableInt64", FieldType.nullable(new ArrowType.Int(64, true)), null),
+ new Field("nonNullableInt64", FieldType.notNullable(new ArrowType.Int(64, true)), null),
+ new Field("nullableUInt64", FieldType.nullable(new ArrowType.Int(64, false)), null),
+ new Field(
+ "nonNullableUInt64", FieldType.notNullable(new ArrowType.Int(64, false)), null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(16, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableInt8").schema().getType());
+ assertEquals(2, schema.getField("nullableInt8").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("nullableInt8").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableInt8").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.INT, schema.getField("nonNullableInt8").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableUInt8").schema().getType());
+ assertEquals(2, schema.getField("nullableUInt8").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("nullableUInt8").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableUInt8").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.INT, schema.getField("nonNullableUInt8").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableInt16").schema().getType());
+ assertEquals(2, schema.getField("nullableInt16").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("nullableInt16").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableInt16").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.INT, schema.getField("nonNullableInt16").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableUInt16").schema().getType());
+ assertEquals(2, schema.getField("nullableUInt16").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("nullableUInt16").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableUInt16").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.INT, schema.getField("nonNullableUInt16").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableInt32").schema().getType());
+ assertEquals(2, schema.getField("nullableInt32").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("nullableInt32").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableInt32").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.INT, schema.getField("nonNullableInt32").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableUInt32").schema().getType());
+ assertEquals(2, schema.getField("nullableUInt32").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.LONG, schema.getField("nullableUInt32").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableUInt32").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.LONG, schema.getField("nonNullableUInt32").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableInt64").schema().getType());
+ assertEquals(2, schema.getField("nullableInt64").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.LONG, schema.getField("nullableInt64").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableInt64").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.LONG, schema.getField("nonNullableInt64").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableUInt64").schema().getType());
+ assertEquals(2, schema.getField("nullableUInt64").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.LONG, schema.getField("nullableUInt64").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableUInt64").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.LONG, schema.getField("nonNullableUInt64").schema().getType());
+ }
+
+ @Test
+ public void testConvertFloatingPointTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableFloat16",
+ FieldType.nullable(new ArrowType.FloatingPoint(FloatingPointPrecision.HALF)),
+ null),
+ new Field(
+ "nonNullableFloat16",
+ FieldType.notNullable(new ArrowType.FloatingPoint(FloatingPointPrecision.HALF)),
+ null),
+ new Field(
+ "nullableFloat32",
+ FieldType.nullable(new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field(
+ "nonNullableFloat32",
+ FieldType.notNullable(new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field(
+ "nullableFloat64",
+ FieldType.nullable(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null),
+ new Field(
+ "nonNullableFloat64",
+ FieldType.notNullable(new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(6, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableFloat16").schema().getType());
+ assertEquals(2, schema.getField("nullableFloat16").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.FLOAT, schema.getField("nullableFloat16").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableFloat16").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.FLOAT, schema.getField("nonNullableFloat16").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableFloat32").schema().getType());
+ assertEquals(2, schema.getField("nullableFloat32").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.FLOAT, schema.getField("nullableFloat32").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableFloat32").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.FLOAT, schema.getField("nonNullableFloat32").schema().getType());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableFloat64").schema().getType());
+ assertEquals(2, schema.getField("nullableFloat64").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.DOUBLE,
+ schema.getField("nullableFloat64").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableFloat64").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.DOUBLE, schema.getField("nonNullableFloat64").schema().getType());
+ }
+
+ @Test
+ public void testConvertStringTypes() {
+ List fields =
+ Arrays.asList(
+ new Field("nullableUtf8", FieldType.nullable(new ArrowType.Utf8()), null),
+ new Field("nonNullableUtf8", FieldType.notNullable(new ArrowType.Utf8()), null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(2, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableUtf8").schema().getType());
+ assertEquals(2, schema.getField("nullableUtf8").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.STRING, schema.getField("nullableUtf8").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableUtf8").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.STRING, schema.getField("nonNullableUtf8").schema().getType());
+ }
+
+ @Test
+ public void testConvertBinaryTypes() {
+ List fields =
+ Arrays.asList(
+ new Field("nullableBinary", FieldType.nullable(new ArrowType.Binary()), null),
+ new Field("nonNullableBinary", FieldType.notNullable(new ArrowType.Binary()), null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(2, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableBinary").schema().getType());
+ assertEquals(2, schema.getField("nullableBinary").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.BYTES, schema.getField("nullableBinary").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableBinary").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.BYTES, schema.getField("nonNullableBinary").schema().getType());
+ }
+
+ @Test
+ public void testConvertFixedSizeBinaryTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableFixedSizeBinary",
+ FieldType.nullable(new ArrowType.FixedSizeBinary(10)),
+ null),
+ new Field(
+ "nonNullableFixedSizeBinary",
+ FieldType.notNullable(new ArrowType.FixedSizeBinary(10)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(2, schema.getFields().size());
+
+ assertEquals(Schema.Type.UNION, schema.getField("nullableFixedSizeBinary").schema().getType());
+ assertEquals(2, schema.getField("nullableFixedSizeBinary").schema().getTypes().size());
+ Schema nullableFixedSizeBinarySchema =
+ schema.getField("nullableFixedSizeBinary").schema().getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableFixedSizeBinarySchema.getType());
+ assertEquals(10, nullableFixedSizeBinarySchema.getFixedSize());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableFixedSizeBinary").schema().getTypes().get(1).getType());
+ Schema nonNullableFixedSizeBinarySchema =
+ schema.getField("nullableFixedSizeBinary").schema().getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nonNullableFixedSizeBinarySchema.getType());
+ assertEquals(10, nonNullableFixedSizeBinarySchema.getFixedSize());
+ }
+
+ // Schema conversion for logical types, nullable and non-nullable
+
+ @Test
+ public void testConvertDecimalTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableDecimal128", FieldType.nullable(new ArrowType.Decimal(10, 2, 128)), null),
+ new Field(
+ "nonNullableDecimal1281",
+ FieldType.notNullable(new ArrowType.Decimal(10, 2, 128)),
+ null),
+ new Field(
+ "nonNullableDecimal1282",
+ FieldType.notNullable(new ArrowType.Decimal(15, 5, 128)),
+ null),
+ new Field(
+ "nonNullableDecimal1283",
+ FieldType.notNullable(new ArrowType.Decimal(20, 10, 128)),
+ null),
+ new Field(
+ "nullableDecimal256", FieldType.nullable(new ArrowType.Decimal(20, 4, 256)), null),
+ new Field(
+ "nonNullableDecimal2561",
+ FieldType.notNullable(new ArrowType.Decimal(20, 4, 256)),
+ null),
+ new Field(
+ "nonNullableDecimal2562",
+ FieldType.notNullable(new ArrowType.Decimal(25, 8, 256)),
+ null),
+ new Field(
+ "nonNullableDecimal2563",
+ FieldType.notNullable(new ArrowType.Decimal(30, 15, 256)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(8, schema.getFields().size());
+
+ // Assertions for nullableDecimal128
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDecimal128").schema().getType());
+ assertEquals(2, schema.getField("nullableDecimal128").schema().getTypes().size());
+ Schema nullableDecimal128Schema =
+ schema.getField("nullableDecimal128").schema().getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableDecimal128Schema.getType());
+ assertEquals(16, nullableDecimal128Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(10, 2), nullableDecimal128Schema.getLogicalType());
+ assertEquals(10, nullableDecimal128Schema.getObjectProp("precision"));
+ assertEquals(2, nullableDecimal128Schema.getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableDecimal128").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableDecimal1281
+ Schema nonNullableDecimal1281Schema = schema.getField("nonNullableDecimal1281").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal1281Schema.getType());
+ assertEquals(16, nonNullableDecimal1281Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(10, 2), nonNullableDecimal1281Schema.getLogicalType());
+ assertEquals(10, nonNullableDecimal1281Schema.getObjectProp("precision"));
+ assertEquals(2, nonNullableDecimal1281Schema.getObjectProp("scale"));
+
+ // Assertions for nonNullableDecimal1282
+ Schema nonNullableDecimal1282Schema = schema.getField("nonNullableDecimal1282").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal1282Schema.getType());
+ assertEquals(16, nonNullableDecimal1282Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(15, 5), nonNullableDecimal1282Schema.getLogicalType());
+ assertEquals(15, nonNullableDecimal1282Schema.getObjectProp("precision"));
+ assertEquals(5, nonNullableDecimal1282Schema.getObjectProp("scale"));
+
+ // Assertions for nonNullableDecimal1283
+ Schema nonNullableDecimal1283Schema = schema.getField("nonNullableDecimal1283").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal1283Schema.getType());
+ assertEquals(16, nonNullableDecimal1283Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(20, 10), nonNullableDecimal1283Schema.getLogicalType());
+ assertEquals(20, nonNullableDecimal1283Schema.getObjectProp("precision"));
+ assertEquals(10, nonNullableDecimal1283Schema.getObjectProp("scale"));
+
+ // Assertions for nullableDecimal256
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDecimal256").schema().getType());
+ assertEquals(2, schema.getField("nullableDecimal256").schema().getTypes().size());
+ Schema nullableDecimal256Schema =
+ schema.getField("nullableDecimal256").schema().getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableDecimal256Schema.getType());
+ assertEquals(32, nullableDecimal256Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(20, 4), nullableDecimal256Schema.getLogicalType());
+ assertEquals(20, nullableDecimal256Schema.getObjectProp("precision"));
+ assertEquals(4, nullableDecimal256Schema.getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableDecimal256").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableDecimal2561
+ Schema nonNullableDecimal2561Schema = schema.getField("nonNullableDecimal2561").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal2561Schema.getType());
+ assertEquals(32, nonNullableDecimal2561Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(20, 4), nonNullableDecimal2561Schema.getLogicalType());
+ assertEquals(20, nonNullableDecimal2561Schema.getObjectProp("precision"));
+ assertEquals(4, nonNullableDecimal2561Schema.getObjectProp("scale"));
+
+ // Assertions for nonNullableDecimal2562
+ Schema nonNullableDecimal2562Schema = schema.getField("nonNullableDecimal2562").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal2562Schema.getType());
+ assertEquals(32, nonNullableDecimal2562Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(25, 8), nonNullableDecimal2562Schema.getLogicalType());
+ assertEquals(25, nonNullableDecimal2562Schema.getObjectProp("precision"));
+ assertEquals(8, nonNullableDecimal2562Schema.getObjectProp("scale"));
+
+ // Assertions for nonNullableDecimal2563
+ Schema nonNullableDecimal2563Schema = schema.getField("nonNullableDecimal2563").schema();
+ assertEquals(Schema.Type.FIXED, nonNullableDecimal2563Schema.getType());
+ assertEquals(32, nonNullableDecimal2563Schema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(30, 15), nonNullableDecimal2563Schema.getLogicalType());
+ assertEquals(30, nonNullableDecimal2563Schema.getObjectProp("precision"));
+ assertEquals(15, nonNullableDecimal2563Schema.getObjectProp("scale"));
+ }
+
+ @Test
+ public void testConvertDateTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableDateDay", FieldType.nullable(new ArrowType.Date(DateUnit.DAY)), null),
+ new Field(
+ "nonNullableDateDay",
+ FieldType.notNullable(new ArrowType.Date(DateUnit.DAY)),
+ null),
+ new Field(
+ "nullableDateMilli",
+ FieldType.nullable(new ArrowType.Date(DateUnit.MILLISECOND)),
+ null),
+ new Field(
+ "nonNullableDateMilli",
+ FieldType.notNullable(new ArrowType.Date(DateUnit.MILLISECOND)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(4, schema.getFields().size());
+
+ // Assertions for nullableDateDay
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDateDay").schema().getType());
+ assertEquals(2, schema.getField("nullableDateDay").schema().getTypes().size());
+ Schema nullableDateDaySchema = schema.getField("nullableDateDay").schema().getTypes().get(0);
+ assertEquals(Schema.Type.INT, nullableDateDaySchema.getType());
+ assertEquals(LogicalTypes.date(), nullableDateDaySchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableDateDay").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableDateDay
+ Schema nonNullableDateDaySchema = schema.getField("nonNullableDateDay").schema();
+ assertEquals(Schema.Type.INT, nonNullableDateDaySchema.getType());
+ assertEquals(LogicalTypes.date(), nonNullableDateDaySchema.getLogicalType());
+
+ // Assertions for nullableDateMilli
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDateMilli").schema().getType());
+ assertEquals(2, schema.getField("nullableDateMilli").schema().getTypes().size());
+ Schema nullableDateMilliSchema =
+ schema.getField("nullableDateMilli").schema().getTypes().get(0);
+ assertEquals(Schema.Type.INT, nullableDateMilliSchema.getType());
+ assertEquals(LogicalTypes.date(), nullableDateMilliSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableDateMilli").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableDateMilli
+ Schema nonNullableDateMilliSchema = schema.getField("nonNullableDateMilli").schema();
+ assertEquals(Schema.Type.INT, nonNullableDateMilliSchema.getType());
+ assertEquals(LogicalTypes.date(), nonNullableDateMilliSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertTimeTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableTimeSec",
+ FieldType.nullable(new ArrowType.Time(TimeUnit.SECOND, 32)),
+ null),
+ new Field(
+ "nonNullableTimeSec",
+ FieldType.notNullable(new ArrowType.Time(TimeUnit.SECOND, 32)),
+ null),
+ new Field(
+ "nullableTimeMillis",
+ FieldType.nullable(new ArrowType.Time(TimeUnit.MILLISECOND, 32)),
+ null),
+ new Field(
+ "nonNullableTimeMillis",
+ FieldType.notNullable(new ArrowType.Time(TimeUnit.MILLISECOND, 32)),
+ null),
+ new Field(
+ "nullableTimeMicros",
+ FieldType.nullable(new ArrowType.Time(TimeUnit.MICROSECOND, 64)),
+ null),
+ new Field(
+ "nonNullableTimeMicros",
+ FieldType.notNullable(new ArrowType.Time(TimeUnit.MICROSECOND, 64)),
+ null),
+ new Field(
+ "nullableTimeNanos",
+ FieldType.nullable(new ArrowType.Time(TimeUnit.NANOSECOND, 64)),
+ null),
+ new Field(
+ "nonNullableTimeNanos",
+ FieldType.notNullable(new ArrowType.Time(TimeUnit.NANOSECOND, 64)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(8, schema.getFields().size());
+
+ // Assertions for nullableTimeSec
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimeSec").schema().getType());
+ assertEquals(2, schema.getField("nullableTimeSec").schema().getTypes().size());
+ Schema nullableTimeSecSchema = schema.getField("nullableTimeSec").schema().getTypes().get(0);
+ assertEquals(Schema.Type.INT, nullableTimeSecSchema.getType());
+ assertEquals(LogicalTypes.timeMillis(), nullableTimeSecSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableTimeSec").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimeSec
+ Schema nonNullableTimeSecSchema = schema.getField("nonNullableTimeSec").schema();
+ assertEquals(Schema.Type.INT, nonNullableTimeSecSchema.getType());
+ assertEquals(LogicalTypes.timeMillis(), nonNullableTimeSecSchema.getLogicalType());
+
+ // Assertions for nullableTimeMillis
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimeMillis").schema().getType());
+ assertEquals(2, schema.getField("nullableTimeMillis").schema().getTypes().size());
+ Schema nullableTimeMillisSchema =
+ schema.getField("nullableTimeMillis").schema().getTypes().get(0);
+ assertEquals(Schema.Type.INT, nullableTimeMillisSchema.getType());
+ assertEquals(LogicalTypes.timeMillis(), nullableTimeMillisSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimeMillis").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimeMillis
+ Schema nonNullableTimeMillisSchema = schema.getField("nonNullableTimeMillis").schema();
+ assertEquals(Schema.Type.INT, nonNullableTimeMillisSchema.getType());
+ assertEquals(LogicalTypes.timeMillis(), nonNullableTimeMillisSchema.getLogicalType());
+
+ // Assertions for nullableTimeMicros
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimeMicros").schema().getType());
+ assertEquals(2, schema.getField("nullableTimeMicros").schema().getTypes().size());
+ Schema nullableTimeMicrosSchema =
+ schema.getField("nullableTimeMicros").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimeMicrosSchema.getType());
+ assertEquals(LogicalTypes.timeMicros(), nullableTimeMicrosSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimeMicros").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimeMicros
+ Schema nonNullableTimeMicrosSchema = schema.getField("nonNullableTimeMicros").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimeMicrosSchema.getType());
+ assertEquals(LogicalTypes.timeMicros(), nonNullableTimeMicrosSchema.getLogicalType());
+
+ // Assertions for nullableTimeNanos
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimeNanos").schema().getType());
+ assertEquals(2, schema.getField("nullableTimeNanos").schema().getTypes().size());
+ Schema nullableTimeNanosSchema =
+ schema.getField("nullableTimeNanos").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimeNanosSchema.getType());
+ assertEquals(LogicalTypes.timeMicros(), nullableTimeNanosSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimeNanos").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimeNanos
+ Schema nonNullableTimeNanosSchema = schema.getField("nonNullableTimeNanos").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimeNanosSchema.getType());
+ assertEquals(LogicalTypes.timeMicros(), nonNullableTimeNanosSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertZoneAwareTimestampTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableTimestampSecTz",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, "UTC")),
+ null),
+ new Field(
+ "nonNullableTimestampSecTz",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.SECOND, "UTC")),
+ null),
+ new Field(
+ "nullableTimestampMillisTz",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null),
+ new Field(
+ "nonNullableTimestampMillisTz",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null),
+ new Field(
+ "nullableTimestampMicrosTz",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC")),
+ null),
+ new Field(
+ "nonNullableTimestampMicrosTz",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MICROSECOND, "UTC")),
+ null),
+ new Field(
+ "nullableTimestampNanosTz",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC")),
+ null),
+ new Field(
+ "nonNullableTimestampNanosTz",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.NANOSECOND, "UTC")),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(8, schema.getFields().size());
+
+ // Assertions for nullableTimestampSecTz
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampSecTz").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampSecTz").schema().getTypes().size());
+ Schema nullableTimestampSecTzSchema =
+ schema.getField("nullableTimestampSecTz").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampSecTzSchema.getType());
+ assertEquals(LogicalTypes.timestampMillis(), nullableTimestampSecTzSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampSecTz").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampSecTz
+ Schema nonNullableTimestampSecTzSchema = schema.getField("nonNullableTimestampSecTz").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampSecTzSchema.getType());
+ assertEquals(LogicalTypes.timestampMillis(), nonNullableTimestampSecTzSchema.getLogicalType());
+
+ // Assertions for nullableTimestampMillisTz
+ assertEquals(
+ Schema.Type.UNION, schema.getField("nullableTimestampMillisTz").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampMillisTz").schema().getTypes().size());
+ Schema nullableTimestampMillisTzSchema =
+ schema.getField("nullableTimestampMillisTz").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampMillisTzSchema.getType());
+ assertEquals(LogicalTypes.timestampMillis(), nullableTimestampMillisTzSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampMillisTz").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampMillisTz
+ Schema nonNullableTimestampMillisTzSchema =
+ schema.getField("nonNullableTimestampMillisTz").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampMillisTzSchema.getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(), nonNullableTimestampMillisTzSchema.getLogicalType());
+
+ // Assertions for nullableTimestampMicrosTz
+ assertEquals(
+ Schema.Type.UNION, schema.getField("nullableTimestampMicrosTz").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampMicrosTz").schema().getTypes().size());
+ Schema nullableTimestampMicrosTzSchema =
+ schema.getField("nullableTimestampMicrosTz").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampMicrosTzSchema.getType());
+ assertEquals(LogicalTypes.timestampMicros(), nullableTimestampMicrosTzSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampMicrosTz").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampMicrosTz
+ Schema nonNullableTimestampMicrosTzSchema =
+ schema.getField("nonNullableTimestampMicrosTz").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampMicrosTzSchema.getType());
+ assertEquals(
+ LogicalTypes.timestampMicros(), nonNullableTimestampMicrosTzSchema.getLogicalType());
+
+ // Assertions for nullableTimestampNanosTz
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampNanosTz").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampNanosTz").schema().getTypes().size());
+ Schema nullableTimestampNanosTzSchema =
+ schema.getField("nullableTimestampNanosTz").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampNanosTzSchema.getType());
+ assertEquals(LogicalTypes.timestampNanos(), nullableTimestampNanosTzSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampNanosTz").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampNanosTz
+ Schema nonNullableTimestampNanosTzSchema =
+ schema.getField("nonNullableTimestampNanosTz").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampNanosTzSchema.getType());
+ assertEquals(LogicalTypes.timestampNanos(), nonNullableTimestampNanosTzSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertLocalTimestampTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableTimestampSec",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.SECOND, null)),
+ null),
+ new Field(
+ "nonNullableTimestampSec",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.SECOND, null)),
+ null),
+ new Field(
+ "nullableTimestampMillis",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, null)),
+ null),
+ new Field(
+ "nonNullableTimestampMillis",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, null)),
+ null),
+ new Field(
+ "nullableTimestampMicros",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.MICROSECOND, null)),
+ null),
+ new Field(
+ "nonNullableTimestampMicros",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MICROSECOND, null)),
+ null),
+ new Field(
+ "nullableTimestampNanos",
+ FieldType.nullable(new ArrowType.Timestamp(TimeUnit.NANOSECOND, null)),
+ null),
+ new Field(
+ "nonNullableTimestampNanos",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.NANOSECOND, null)),
+ null));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(8, schema.getFields().size());
+
+ // Assertions for nullableTimestampSec
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampSec").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampSec").schema().getTypes().size());
+ Schema nullableTimestampSecSchema =
+ schema.getField("nullableTimestampSec").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampSecSchema.getType());
+ assertEquals(LogicalTypes.localTimestampMillis(), nullableTimestampSecSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampSec").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampSec
+ Schema nonNullableTimestampSecSchema = schema.getField("nonNullableTimestampSec").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampSecSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampMillis(), nonNullableTimestampSecSchema.getLogicalType());
+
+ // Assertions for nullableTimestampMillis
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampMillis").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampMillis").schema().getTypes().size());
+ Schema nullableTimestampMillisSchema =
+ schema.getField("nullableTimestampMillis").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampMillisSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampMillis(), nullableTimestampMillisSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampMillis").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampMillis
+ Schema nonNullableTimestampMillisSchema =
+ schema.getField("nonNullableTimestampMillis").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampMillisSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampMillis(), nonNullableTimestampMillisSchema.getLogicalType());
+
+ // Assertions for nullableTimestampMicros
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampMicros").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampMicros").schema().getTypes().size());
+ Schema nullableTimestampMicrosSchema =
+ schema.getField("nullableTimestampMicros").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampMicrosSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampMicros(), nullableTimestampMicrosSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampMicros").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampMicros
+ Schema nonNullableTimestampMicrosSchema =
+ schema.getField("nonNullableTimestampMicros").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampMicrosSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampMicros(), nonNullableTimestampMicrosSchema.getLogicalType());
+
+ // Assertions for nullableTimestampNanos
+ assertEquals(Schema.Type.UNION, schema.getField("nullableTimestampNanos").schema().getType());
+ assertEquals(2, schema.getField("nullableTimestampNanos").schema().getTypes().size());
+ Schema nullableTimestampNanosSchema =
+ schema.getField("nullableTimestampNanos").schema().getTypes().get(0);
+ assertEquals(Schema.Type.LONG, nullableTimestampNanosSchema.getType());
+ assertEquals(LogicalTypes.localTimestampNanos(), nullableTimestampNanosSchema.getLogicalType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableTimestampNanos").schema().getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampNanos
+ Schema nonNullableTimestampNanosSchema = schema.getField("nonNullableTimestampNanos").schema();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampNanosSchema.getType());
+ assertEquals(
+ LogicalTypes.localTimestampNanos(), nonNullableTimestampNanosSchema.getLogicalType());
+ }
+
+ // Schema conversion for complex types, where the contents are primitive and logical types
+
+ @Test
+ public void testConvertListTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableIntList",
+ FieldType.nullable(new ArrowType.List()),
+ Arrays.asList(
+ new Field("item", FieldType.nullable(new ArrowType.Int(32, true)), null))),
+ new Field(
+ "nullableDoubleList",
+ FieldType.nullable(new ArrowType.List()),
+ Arrays.asList(
+ new Field(
+ "item",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null))),
+ new Field(
+ "nonNullableDecimalList",
+ FieldType.notNullable(new ArrowType.List()),
+ Arrays.asList(
+ new Field(
+ "item", FieldType.nullable(new ArrowType.Decimal(10, 2, 128)), null))),
+ new Field(
+ "nonNullableTimestampList",
+ FieldType.notNullable(new ArrowType.List()),
+ Arrays.asList(
+ new Field(
+ "item",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null))));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(4, schema.getFields().size());
+
+ // Assertions for nullableIntList
+ assertEquals(Schema.Type.UNION, schema.getField("nullableIntList").schema().getType());
+ assertEquals(2, schema.getField("nullableIntList").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.ARRAY, schema.getField("nullableIntList").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableIntList").schema().getTypes().get(1).getType());
+ Schema nullableIntListItemSchema =
+ schema.getField("nullableIntList").schema().getTypes().get(0).getElementType();
+ assertEquals(Schema.Type.UNION, nullableIntListItemSchema.getType());
+ assertEquals(2, nullableIntListItemSchema.getTypes().size());
+ assertEquals(Schema.Type.INT, nullableIntListItemSchema.getTypes().get(0).getType());
+ assertEquals(Schema.Type.NULL, nullableIntListItemSchema.getTypes().get(1).getType());
+
+ // Assertions for nullableDoubleList
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDoubleList").schema().getType());
+ assertEquals(2, schema.getField("nullableDoubleList").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.ARRAY,
+ schema.getField("nullableDoubleList").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableDoubleList").schema().getTypes().get(1).getType());
+ Schema nullableDoubleListItemSchema =
+ schema.getField("nullableDoubleList").schema().getTypes().get(0).getElementType();
+ assertEquals(Schema.Type.DOUBLE, nullableDoubleListItemSchema.getType());
+
+ // Assertions for nonNullableDecimalList
+ assertEquals(Schema.Type.ARRAY, schema.getField("nonNullableDecimalList").schema().getType());
+ Schema nonNullableDecimalListItemSchema =
+ schema.getField("nonNullableDecimalList").schema().getElementType();
+ assertEquals(Schema.Type.UNION, nonNullableDecimalListItemSchema.getType());
+ assertEquals(2, nonNullableDecimalListItemSchema.getTypes().size());
+ Schema nullableDecimalSchema = nonNullableDecimalListItemSchema.getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableDecimalSchema.getType());
+ assertEquals(16, nullableDecimalSchema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(10, 2), nullableDecimalSchema.getLogicalType());
+ assertEquals(10, nullableDecimalSchema.getObjectProp("precision"));
+ assertEquals(2, nullableDecimalSchema.getObjectProp("scale"));
+ assertEquals(Schema.Type.NULL, nonNullableDecimalListItemSchema.getTypes().get(1).getType());
+
+ // Assertions for nonNullableTimestampList
+ assertEquals(Schema.Type.ARRAY, schema.getField("nonNullableTimestampList").schema().getType());
+ Schema nonNullableTimestampListItemSchema =
+ schema.getField("nonNullableTimestampList").schema().getElementType();
+ assertEquals(Schema.Type.LONG, nonNullableTimestampListItemSchema.getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(), nonNullableTimestampListItemSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertFixedSizeListTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableFixedSizeIntList",
+ FieldType.nullable(new ArrowType.FixedSizeList(3)),
+ Arrays.asList(
+ new Field("item", FieldType.nullable(new ArrowType.Int(32, true)), null))),
+ new Field(
+ "nullableFixedSizeDoubleList",
+ FieldType.nullable(new ArrowType.FixedSizeList(3)),
+ Arrays.asList(
+ new Field(
+ "item",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null))),
+ new Field(
+ "nonNullableFixedSizeDecimalList",
+ FieldType.notNullable(new ArrowType.FixedSizeList(3)),
+ Arrays.asList(
+ new Field(
+ "item", FieldType.nullable(new ArrowType.Decimal(10, 2, 128)), null))),
+ new Field(
+ "nonNullableFixedSizeTimestampList",
+ FieldType.notNullable(new ArrowType.FixedSizeList(3)),
+ Arrays.asList(
+ new Field(
+ "item",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null))));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(4, schema.getFields().size());
+
+ // Assertions for nullableFixedSizeIntList
+ assertEquals(Schema.Type.UNION, schema.getField("nullableFixedSizeIntList").schema().getType());
+ assertEquals(2, schema.getField("nullableFixedSizeIntList").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.ARRAY,
+ schema.getField("nullableFixedSizeIntList").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableFixedSizeIntList").schema().getTypes().get(1).getType());
+ Schema nullableFixedSizeIntListItemSchema =
+ schema.getField("nullableFixedSizeIntList").schema().getTypes().get(0).getElementType();
+ assertEquals(Schema.Type.UNION, nullableFixedSizeIntListItemSchema.getType());
+ assertEquals(2, nullableFixedSizeIntListItemSchema.getTypes().size());
+ assertEquals(Schema.Type.INT, nullableFixedSizeIntListItemSchema.getTypes().get(0).getType());
+ assertEquals(Schema.Type.NULL, nullableFixedSizeIntListItemSchema.getTypes().get(1).getType());
+
+ // Assertions for nullableFixedSizeDoubleList
+ assertEquals(
+ Schema.Type.UNION, schema.getField("nullableFixedSizeDoubleList").schema().getType());
+ assertEquals(2, schema.getField("nullableFixedSizeDoubleList").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.ARRAY,
+ schema.getField("nullableFixedSizeDoubleList").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableFixedSizeDoubleList").schema().getTypes().get(1).getType());
+ Schema nullableFixedSizeDoubleListItemSchema =
+ schema.getField("nullableFixedSizeDoubleList").schema().getTypes().get(0).getElementType();
+ assertEquals(Schema.Type.DOUBLE, nullableFixedSizeDoubleListItemSchema.getType());
+
+ // Assertions for nonNullableFixedSizeDecimalList
+ assertEquals(
+ Schema.Type.ARRAY, schema.getField("nonNullableFixedSizeDecimalList").schema().getType());
+ Schema nonNullableFixedSizeDecimalListItemSchema =
+ schema.getField("nonNullableFixedSizeDecimalList").schema().getElementType();
+ assertEquals(Schema.Type.UNION, nonNullableFixedSizeDecimalListItemSchema.getType());
+ assertEquals(2, nonNullableFixedSizeDecimalListItemSchema.getTypes().size());
+ Schema nullableDecimalSchema = nonNullableFixedSizeDecimalListItemSchema.getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableDecimalSchema.getType());
+ assertEquals(16, nullableDecimalSchema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(10, 2), nullableDecimalSchema.getLogicalType());
+ assertEquals(10, nullableDecimalSchema.getObjectProp("precision"));
+ assertEquals(2, nullableDecimalSchema.getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL, nonNullableFixedSizeDecimalListItemSchema.getTypes().get(1).getType());
+
+ // Assertions for nonNullableFixedSizeTimestampList
+ assertEquals(
+ Schema.Type.ARRAY, schema.getField("nonNullableFixedSizeTimestampList").schema().getType());
+ Schema nonNullableFixedSizeTimestampListItemSchema =
+ schema.getField("nonNullableFixedSizeTimestampList").schema().getElementType();
+ assertEquals(Schema.Type.LONG, nonNullableFixedSizeTimestampListItemSchema.getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(),
+ nonNullableFixedSizeTimestampListItemSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertMapTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableMapWithNullableInt",
+ FieldType.nullable(new ArrowType.Map(false)),
+ Arrays.asList(
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null),
+ new Field(
+ "value", FieldType.nullable(new ArrowType.Int(32, true)), null))))),
+ new Field(
+ "nullableMapWithNonNullableDouble",
+ FieldType.nullable(new ArrowType.Map(false)),
+ Arrays.asList(
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null),
+ new Field(
+ "value",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null))))),
+ new Field(
+ "nonNullableMapWithNullableDecimal",
+ FieldType.notNullable(new ArrowType.Map(false)),
+ Arrays.asList(
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null),
+ new Field(
+ "value",
+ FieldType.nullable(new ArrowType.Decimal(10, 2, 128)),
+ null))))),
+ new Field(
+ "nonNullableMapWithNonNullableTimestamp",
+ FieldType.notNullable(new ArrowType.Map(false)),
+ Arrays.asList(
+ new Field(
+ "entries",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("key", FieldType.notNullable(new ArrowType.Utf8()), null),
+ new Field(
+ "value",
+ FieldType.notNullable(
+ new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null))))));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(4, schema.getFields().size());
+
+ // Assertions for nullableMapWithNullableInt
+ assertEquals(
+ Schema.Type.UNION, schema.getField("nullableMapWithNullableInt").schema().getType());
+ assertEquals(2, schema.getField("nullableMapWithNullableInt").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.MAP,
+ schema.getField("nullableMapWithNullableInt").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableMapWithNullableInt").schema().getTypes().get(1).getType());
+ Schema nullableMapWithNullableIntValueSchema =
+ schema.getField("nullableMapWithNullableInt").schema().getTypes().get(0).getValueType();
+ assertEquals(Schema.Type.UNION, nullableMapWithNullableIntValueSchema.getType());
+ assertEquals(2, nullableMapWithNullableIntValueSchema.getTypes().size());
+ assertEquals(
+ Schema.Type.INT, nullableMapWithNullableIntValueSchema.getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, nullableMapWithNullableIntValueSchema.getTypes().get(1).getType());
+
+ // Assertions for nullableMapWithNonNullableDouble
+ assertEquals(
+ Schema.Type.UNION, schema.getField("nullableMapWithNonNullableDouble").schema().getType());
+ assertEquals(2, schema.getField("nullableMapWithNonNullableDouble").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.MAP,
+ schema.getField("nullableMapWithNonNullableDouble").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableMapWithNonNullableDouble").schema().getTypes().get(1).getType());
+ Schema nullableMapWithNonNullableDoubleValueSchema =
+ schema
+ .getField("nullableMapWithNonNullableDouble")
+ .schema()
+ .getTypes()
+ .get(0)
+ .getValueType();
+ assertEquals(Schema.Type.DOUBLE, nullableMapWithNonNullableDoubleValueSchema.getType());
+
+ // Assertions for nonNullableMapWithNullableDecimal
+ assertEquals(
+ Schema.Type.MAP, schema.getField("nonNullableMapWithNullableDecimal").schema().getType());
+ Schema nonNullableMapWithNullableDecimalValueSchema =
+ schema.getField("nonNullableMapWithNullableDecimal").schema().getValueType();
+ assertEquals(Schema.Type.UNION, nonNullableMapWithNullableDecimalValueSchema.getType());
+ assertEquals(2, nonNullableMapWithNullableDecimalValueSchema.getTypes().size());
+ Schema nullableDecimalSchema = nonNullableMapWithNullableDecimalValueSchema.getTypes().get(0);
+ assertEquals(Schema.Type.FIXED, nullableDecimalSchema.getType());
+ assertEquals(16, nullableDecimalSchema.getFixedSize());
+ assertEquals(LogicalTypes.decimal(10, 2), nullableDecimalSchema.getLogicalType());
+ assertEquals(10, nullableDecimalSchema.getObjectProp("precision"));
+ assertEquals(2, nullableDecimalSchema.getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL, nonNullableMapWithNullableDecimalValueSchema.getTypes().get(1).getType());
+
+ // Assertions for nonNullableMapWithNonNullableTimestamp
+ assertEquals(
+ Schema.Type.MAP,
+ schema.getField("nonNullableMapWithNonNullableTimestamp").schema().getType());
+ Schema nonNullableMapWithNonNullableTimestampValueSchema =
+ schema.getField("nonNullableMapWithNonNullableTimestamp").schema().getValueType();
+ assertEquals(Schema.Type.LONG, nonNullableMapWithNonNullableTimestampValueSchema.getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(),
+ nonNullableMapWithNonNullableTimestampValueSchema.getLogicalType());
+ }
+
+ @Test
+ public void testConvertRecordTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "nullableRecord",
+ FieldType.nullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("field1", FieldType.nullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "field2",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null),
+ new Field(
+ "field3", FieldType.nullable(new ArrowType.Decimal(10, 2, 128)), null),
+ new Field(
+ "field4",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null))),
+ new Field(
+ "nonNullableRecord",
+ FieldType.notNullable(new ArrowType.Struct()),
+ Arrays.asList(
+ new Field("field1", FieldType.nullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "field2",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.DOUBLE)),
+ null),
+ new Field(
+ "field3", FieldType.nullable(new ArrowType.Decimal(10, 2, 128)), null),
+ new Field(
+ "field4",
+ FieldType.notNullable(new ArrowType.Timestamp(TimeUnit.MILLISECOND, "UTC")),
+ null))));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(2, schema.getFields().size());
+
+ // Assertions for nullableRecord
+ assertEquals(Schema.Type.UNION, schema.getField("nullableRecord").schema().getType());
+ assertEquals(2, schema.getField("nullableRecord").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.RECORD, schema.getField("nullableRecord").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL, schema.getField("nullableRecord").schema().getTypes().get(1).getType());
+ Schema nullableRecordSchema = schema.getField("nullableRecord").schema().getTypes().get(0);
+ assertEquals(4, nullableRecordSchema.getFields().size());
+ assertEquals(
+ Schema.Type.INT,
+ nullableRecordSchema.getField("field1").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ nullableRecordSchema.getField("field1").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.DOUBLE, nullableRecordSchema.getField("field2").schema().getType());
+ assertEquals(
+ Schema.Type.FIXED,
+ nullableRecordSchema.getField("field3").schema().getTypes().get(0).getType());
+ assertEquals(
+ 16, nullableRecordSchema.getField("field3").schema().getTypes().get(0).getFixedSize());
+ assertEquals(
+ LogicalTypes.decimal(10, 2),
+ nullableRecordSchema.getField("field3").schema().getTypes().get(0).getLogicalType());
+ assertEquals(
+ 10,
+ nullableRecordSchema
+ .getField("field3")
+ .schema()
+ .getTypes()
+ .get(0)
+ .getObjectProp("precision"));
+ assertEquals(
+ 2,
+ nullableRecordSchema.getField("field3").schema().getTypes().get(0).getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL,
+ nullableRecordSchema.getField("field3").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.LONG, nullableRecordSchema.getField("field4").schema().getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(),
+ nullableRecordSchema.getField("field4").schema().getLogicalType());
+
+ // Assertions for nonNullableRecord
+ assertEquals(Schema.Type.RECORD, schema.getField("nonNullableRecord").schema().getType());
+ Schema nonNullableRecordSchema = schema.getField("nonNullableRecord").schema();
+ assertEquals(4, nonNullableRecordSchema.getFields().size());
+ assertEquals(
+ Schema.Type.INT,
+ nonNullableRecordSchema.getField("field1").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.NULL,
+ nonNullableRecordSchema.getField("field1").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.DOUBLE, nonNullableRecordSchema.getField("field2").schema().getType());
+ assertEquals(
+ Schema.Type.FIXED,
+ nonNullableRecordSchema.getField("field3").schema().getTypes().get(0).getType());
+ assertEquals(
+ 16, nullableRecordSchema.getField("field3").schema().getTypes().get(0).getFixedSize());
+ assertEquals(
+ LogicalTypes.decimal(10, 2),
+ nonNullableRecordSchema.getField("field3").schema().getTypes().get(0).getLogicalType());
+ assertEquals(
+ 10,
+ nonNullableRecordSchema
+ .getField("field3")
+ .schema()
+ .getTypes()
+ .get(0)
+ .getObjectProp("precision"));
+ assertEquals(
+ 2,
+ nonNullableRecordSchema
+ .getField("field3")
+ .schema()
+ .getTypes()
+ .get(0)
+ .getObjectProp("scale"));
+ assertEquals(
+ Schema.Type.NULL,
+ nonNullableRecordSchema.getField("field3").schema().getTypes().get(1).getType());
+ assertEquals(Schema.Type.LONG, nonNullableRecordSchema.getField("field4").schema().getType());
+ assertEquals(
+ LogicalTypes.timestampMillis(),
+ nonNullableRecordSchema.getField("field4").schema().getLogicalType());
+ }
+
+ @Test
+ public void testConvertUnionTypes() {
+ List fields =
+ Arrays.asList(
+ new Field(
+ "sparseUnionField",
+ FieldType.nullable(
+ new ArrowType.Union(
+ UnionMode.Sparse,
+ new int[] {
+ ArrowType.ArrowTypeID.Int.getFlatbufID(),
+ ArrowType.ArrowTypeID.FloatingPoint.getFlatbufID(),
+ ArrowType.ArrowTypeID.Utf8.getFlatbufID()
+ })),
+ Arrays.asList(
+ new Field(
+ "intMember", FieldType.notNullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "floatMember",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field("stringMember", FieldType.notNullable(new ArrowType.Utf8()), null))),
+ new Field(
+ "denseUnionField",
+ FieldType.nullable(
+ new ArrowType.Union(
+ UnionMode.Dense,
+ new int[] {
+ ArrowType.ArrowTypeID.Int.getFlatbufID(),
+ ArrowType.ArrowTypeID.FloatingPoint.getFlatbufID(),
+ ArrowType.ArrowTypeID.Utf8.getFlatbufID()
+ })),
+ Arrays.asList(
+ new Field(
+ "intMember", FieldType.notNullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "floatMember",
+ FieldType.notNullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field("stringMember", FieldType.notNullable(new ArrowType.Utf8()), null))),
+ new Field(
+ "nullableSparseUnionField",
+ FieldType.nullable(
+ new ArrowType.Union(
+ UnionMode.Sparse,
+ new int[] {
+ ArrowType.ArrowTypeID.Int.getFlatbufID(),
+ ArrowType.ArrowTypeID.FloatingPoint.getFlatbufID(),
+ ArrowType.ArrowTypeID.Utf8.getFlatbufID()
+ })),
+ Arrays.asList(
+ new Field(
+ "nullableIntMember", FieldType.nullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "nullableFloatMember",
+ FieldType.nullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field(
+ "nullableStringMember", FieldType.nullable(new ArrowType.Utf8()), null))),
+ new Field(
+ "nullableDenseUnionField",
+ FieldType.nullable(
+ new ArrowType.Union(
+ UnionMode.Dense,
+ new int[] {
+ ArrowType.ArrowTypeID.Int.getFlatbufID(),
+ ArrowType.ArrowTypeID.FloatingPoint.getFlatbufID(),
+ ArrowType.ArrowTypeID.Utf8.getFlatbufID()
+ })),
+ Arrays.asList(
+ new Field(
+ "nullableIntMember", FieldType.nullable(new ArrowType.Int(32, true)), null),
+ new Field(
+ "nullableFloatMember",
+ FieldType.nullable(
+ new ArrowType.FloatingPoint(FloatingPointPrecision.SINGLE)),
+ null),
+ new Field(
+ "nullableStringMember", FieldType.nullable(new ArrowType.Utf8()), null))));
+
+ Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord");
+
+ assertEquals(Schema.Type.RECORD, schema.getType());
+ assertEquals(4, schema.getFields().size());
+
+ // Assertions for sparseUnionField
+ assertEquals(Schema.Type.UNION, schema.getField("sparseUnionField").schema().getType());
+ assertEquals(3, schema.getField("sparseUnionField").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("sparseUnionField").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.FLOAT,
+ schema.getField("sparseUnionField").schema().getTypes().get(1).getType());
+ assertEquals(
+ Schema.Type.STRING,
+ schema.getField("sparseUnionField").schema().getTypes().get(2).getType());
+
+ // Assertions for denseUnionField
+ assertEquals(Schema.Type.UNION, schema.getField("denseUnionField").schema().getType());
+ assertEquals(3, schema.getField("denseUnionField").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.INT, schema.getField("denseUnionField").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.FLOAT, schema.getField("denseUnionField").schema().getTypes().get(1).getType());
+ assertEquals(
+ Schema.Type.STRING,
+ schema.getField("denseUnionField").schema().getTypes().get(2).getType());
+
+ // Assertions for sparseUnionField
+ assertEquals(Schema.Type.UNION, schema.getField("nullableSparseUnionField").schema().getType());
+ assertEquals(4, schema.getField("nullableSparseUnionField").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableSparseUnionField").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.INT,
+ schema.getField("nullableSparseUnionField").schema().getTypes().get(1).getType());
+ assertEquals(
+ Schema.Type.FLOAT,
+ schema.getField("nullableSparseUnionField").schema().getTypes().get(2).getType());
+ assertEquals(
+ Schema.Type.STRING,
+ schema.getField("nullableSparseUnionField").schema().getTypes().get(3).getType());
+
+ // Assertions for denseUnionField
+ assertEquals(Schema.Type.UNION, schema.getField("nullableDenseUnionField").schema().getType());
+ assertEquals(4, schema.getField("nullableDenseUnionField").schema().getTypes().size());
+ assertEquals(
+ Schema.Type.NULL,
+ schema.getField("nullableDenseUnionField").schema().getTypes().get(0).getType());
+ assertEquals(
+ Schema.Type.INT,
+ schema.getField("nullableDenseUnionField").schema().getTypes().get(1).getType());
+ assertEquals(
+ Schema.Type.FLOAT,
+ schema.getField("nullableDenseUnionField").schema().getTypes().get(2).getType());
+ assertEquals(
+ Schema.Type.STRING,
+ schema.getField("nullableDenseUnionField").schema().getTypes().get(3).getType());
+ }
+}
diff --git a/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/TestWriteReadAvroRecord.java b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/TestWriteReadAvroRecord.java
index c318214f5..76e58a75a 100644
--- a/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/TestWriteReadAvroRecord.java
+++ b/adapter/avro/src/test/java/org/apache/arrow/adapter/avro/TestWriteReadAvroRecord.java
@@ -19,8 +19,22 @@
import static org.junit.jupiter.api.Assertions.assertEquals;
import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileOutputStream;
+import java.nio.charset.StandardCharsets;
import java.util.ArrayList;
import java.util.List;
+import org.apache.arrow.adapter.avro.consumers.CompositeAvroConsumer;
+import org.apache.arrow.adapter.avro.producers.CompositeAvroProducer;
+import org.apache.arrow.memory.BufferAllocator;
+import org.apache.arrow.memory.RootAllocator;
+import org.apache.arrow.vector.FieldVector;
+import org.apache.arrow.vector.IntVector;
+import org.apache.arrow.vector.VarCharVector;
+import org.apache.arrow.vector.VectorSchemaRoot;
+import org.apache.arrow.vector.types.pojo.ArrowType;
+import org.apache.arrow.vector.types.pojo.Field;
+import org.apache.arrow.vector.types.pojo.FieldType;
import org.apache.avro.Schema;
import org.apache.avro.file.DataFileReader;
import org.apache.avro.file.DataFileWriter;
@@ -28,10 +42,16 @@
import org.apache.avro.generic.GenericDatumReader;
import org.apache.avro.generic.GenericDatumWriter;
import org.apache.avro.generic.GenericRecord;
+import org.apache.avro.io.BinaryDecoder;
+import org.apache.avro.io.BinaryEncoder;
import org.apache.avro.io.DatumReader;
import org.apache.avro.io.DatumWriter;
+import org.apache.avro.io.DecoderFactory;
+import org.apache.avro.io.EncoderFactory;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.io.TempDir;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
public class TestWriteReadAvroRecord {
@@ -82,4 +102,85 @@ public void testWriteAndRead() throws Exception {
assertEquals(7, deUser2.get("favorite_number"));
assertEquals("red", deUser2.get("favorite_color").toString());
}
+
+ @ParameterizedTest
+ @ValueSource(booleans = {true, false})
+ public void testWriteAndReadVSR(boolean useSchemaFile) throws Exception {
+
+ BufferAllocator allocator = new RootAllocator();
+ FieldType stringNotNull = new FieldType(false, ArrowType.Utf8.INSTANCE, null);
+ FieldType stringNull = new FieldType(true, ArrowType.Utf8.INSTANCE, null);
+ FieldType intN32Null = new FieldType(true, new ArrowType.Int(32, true), null);
+
+ List fields = new ArrayList<>();
+ fields.add(new Field("name", stringNotNull, null));
+ fields.add(new Field("favorite_number", intN32Null, null));
+ fields.add(new Field("favorite_color", stringNull, null));
+
+ VarCharVector nameVector = new VarCharVector(fields.get(0), allocator);
+ nameVector.allocateNew(2);
+ nameVector.set(0, "Alyssa".getBytes(StandardCharsets.UTF_8));
+ nameVector.set(1, "Ben".getBytes(StandardCharsets.UTF_8));
+
+ IntVector favNumberVector = new IntVector(fields.get(1), allocator);
+ favNumberVector.allocateNew(2);
+ favNumberVector.set(0, 256);
+ favNumberVector.set(1, 7);
+
+ VarCharVector favColorVector = new VarCharVector(fields.get(2), allocator);
+ favColorVector.allocateNew(2);
+ favColorVector.setNull(0);
+ favColorVector.set(1, "red".getBytes(StandardCharsets.UTF_8));
+
+ List vectors = new ArrayList<>();
+ vectors.add(nameVector);
+ vectors.add(favNumberVector);
+ vectors.add(favColorVector);
+
+ Schema schema =
+ useSchemaFile
+ ? AvroTestBase.getSchema("test.avsc")
+ : ArrowToAvroUtils.createAvroSchema(fields);
+
+ File dataFile = new File(TMP, "test_vsr.avro");
+ AvroToArrowConfig config = new AvroToArrowConfigBuilder(allocator).build();
+
+ try (FileOutputStream fos = new FileOutputStream(dataFile)) {
+
+ BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null);
+ CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors);
+
+ producer.produce(encoder);
+ producer.produce(encoder);
+
+ encoder.flush();
+ }
+
+ List roundTripFields = new ArrayList<>();
+ List roundTripVectors = new ArrayList<>();
+
+ try (FileInputStream fis = new FileInputStream(dataFile)) {
+
+ BinaryDecoder decoder = new DecoderFactory().directBinaryDecoder(fis, null);
+ CompositeAvroConsumer consumer = AvroToArrowUtils.createCompositeConsumer(schema, config);
+
+ consumer.getConsumers().forEach(c -> roundTripFields.add(c.getVector().getField()));
+ consumer.getConsumers().forEach(c -> roundTripVectors.add(c.getVector()));
+ consumer.consume(decoder);
+ consumer.consume(decoder);
+ }
+
+ VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, 2);
+ VectorSchemaRoot roundTripRoot = new VectorSchemaRoot(roundTripFields, roundTripVectors, 2);
+
+ assertEquals(root.getRowCount(), roundTripRoot.getRowCount());
+
+ for (int row = 0; row < 2; row++) {
+ for (int col = 0; col < 3; col++) {
+ FieldVector vector = root.getVector(col);
+ FieldVector roundTripVector = roundTripRoot.getVector(col);
+ assertEquals(vector.getObject(row), roundTripVector.getObject(row));
+ }
+ }
+ }
}