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 typeAvro encoding
ArrowType.NullNULL
ArrowType.BoolBOOLEAN
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.Utf8STRING
ArrowType.LargeUtf8STRING
ArrowType.BinaryBYTES
ArrowType.LargeBinaryBYTES
ArrowType.FixedSizeBinaryFIXED
ArrowType.Decimaldecimal (FIXED)
ArrowType.Datedate (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.Durationduration (FIXED)
ArrowType.Intervalduration (FIXED)
ArrowType.Structrecord
ArrowType.Listarray
ArrowType.LargeListarray
ArrowType.FixedSizeListarray
ArrowType.Mapmap
ArrowType.Unionunion
+ * + *

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 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 delegate; + + /** Instantiate an AvroFixedSizeListProducer. */ + public AvroFixedSizeListProducer( + FixedSizeListVector vector, Producer 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 delegate; + + /** Instantiate an AvroListProducer. */ + public AvroListProducer(ListVector vector, Producer 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 delegate; + + /** Instantiate a AvroMapProducer. */ + public AvroMapProducer(MapVector vector, Producer 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[] delegates; + + /** Instantiate a AvroStructProducer. */ + public AvroStructProducer(StructVector vector, Producer[] 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 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)); + } + } + } }