|
16 | 16 | */ |
17 | 17 | package org.apache.arrow.adapter.avro; |
18 | 18 |
|
19 | | -import static org.junit.jupiter.api.Assertions.assertEquals; |
20 | | - |
21 | 19 | import java.io.File; |
| 20 | +import java.io.FileInputStream; |
| 21 | +import java.io.FileOutputStream; |
| 22 | +import java.nio.charset.StandardCharsets; |
22 | 23 | import java.util.ArrayList; |
23 | 24 | import java.util.List; |
| 25 | + |
| 26 | +import org.apache.arrow.adapter.avro.consumers.CompositeAvroConsumer; |
| 27 | +import org.apache.arrow.adapter.avro.producers.CompositeAvroProducer; |
| 28 | +import org.apache.arrow.memory.BufferAllocator; |
| 29 | +import org.apache.arrow.memory.RootAllocator; |
| 30 | +import org.apache.arrow.vector.FieldVector; |
| 31 | +import org.apache.arrow.vector.IntVector; |
| 32 | +import org.apache.arrow.vector.VarCharVector; |
| 33 | +import org.apache.arrow.vector.VectorSchemaRoot; |
| 34 | +import org.apache.arrow.vector.types.pojo.ArrowType; |
| 35 | +import org.apache.arrow.vector.types.pojo.Field; |
| 36 | +import org.apache.arrow.vector.types.pojo.FieldType; |
24 | 37 | import org.apache.avro.Schema; |
25 | 38 | import org.apache.avro.file.DataFileReader; |
26 | 39 | import org.apache.avro.file.DataFileWriter; |
27 | 40 | import org.apache.avro.generic.GenericData; |
28 | 41 | import org.apache.avro.generic.GenericDatumReader; |
29 | 42 | import org.apache.avro.generic.GenericDatumWriter; |
30 | 43 | import org.apache.avro.generic.GenericRecord; |
| 44 | +import org.apache.avro.io.BinaryDecoder; |
| 45 | +import org.apache.avro.io.BinaryEncoder; |
31 | 46 | import org.apache.avro.io.DatumReader; |
32 | 47 | import org.apache.avro.io.DatumWriter; |
| 48 | +import org.apache.avro.io.DecoderFactory; |
| 49 | +import org.apache.avro.io.EncoderFactory; |
33 | 50 | import org.junit.jupiter.api.Test; |
34 | 51 | import org.junit.jupiter.api.io.TempDir; |
35 | 52 |
|
| 53 | +import static org.junit.jupiter.api.Assertions.*; |
| 54 | + |
36 | 55 | public class TestWriteReadAvroRecord { |
37 | 56 |
|
38 | 57 | @TempDir public static File TMP; |
@@ -82,4 +101,75 @@ public void testWriteAndRead() throws Exception { |
82 | 101 | assertEquals(7, deUser2.get("favorite_number")); |
83 | 102 | assertEquals("red", deUser2.get("favorite_color").toString()); |
84 | 103 | } |
| 104 | + |
| 105 | + @Test |
| 106 | + public void testWriteAndReadVSR() throws Exception { |
| 107 | + |
| 108 | + BufferAllocator allocator = new RootAllocator(); |
| 109 | + |
| 110 | + List<Field> fields = new ArrayList<>(); |
| 111 | + fields.add(new Field("name", new FieldType(false, ArrowType.Utf8.INSTANCE, null), null)); |
| 112 | + fields.add(new Field("favorite_number", new FieldType(true, new ArrowType.Int(32, true), null), null)); |
| 113 | + fields.add(new Field("favorite_color", new FieldType(true, ArrowType.Utf8.INSTANCE, null), null)); |
| 114 | + |
| 115 | + VarCharVector nameVector = new VarCharVector(fields.get(0), allocator); |
| 116 | + nameVector.allocateNew(2); |
| 117 | + nameVector.set(0, "Alyssa".getBytes(StandardCharsets.UTF_8)); |
| 118 | + nameVector.set(1, "Ben".getBytes(StandardCharsets.UTF_8)); |
| 119 | + |
| 120 | + IntVector favNumberVector = new IntVector(fields.get(1), allocator); |
| 121 | + favNumberVector.allocateNew(2); |
| 122 | + favNumberVector.set(0, 256); |
| 123 | + favNumberVector.set(1, 7); |
| 124 | + |
| 125 | + VarCharVector favColorVector = new VarCharVector(fields.get(2), allocator); |
| 126 | + favColorVector.allocateNew(2); |
| 127 | + favColorVector.setNull(0); |
| 128 | + favColorVector.set(1, "red".getBytes(StandardCharsets.UTF_8)); |
| 129 | + |
| 130 | + List<FieldVector> vectors = new ArrayList<>(); |
| 131 | + vectors.add(nameVector); |
| 132 | + vectors.add(favNumberVector); |
| 133 | + vectors.add(favColorVector); |
| 134 | + |
| 135 | + File dataFile = new File(TMP, "test_vsr.avro"); |
| 136 | + Schema schema = AvroTestBase.getSchema("test.avsc"); |
| 137 | + AvroToArrowConfig config = new AvroToArrowConfigBuilder(allocator).build(); |
| 138 | + |
| 139 | + try (FileOutputStream fos = new FileOutputStream(dataFile)) { |
| 140 | + |
| 141 | + BinaryEncoder encoder = new EncoderFactory().directBinaryEncoder(fos, null); |
| 142 | + CompositeAvroProducer producer = ArrowToAvroUtils.createCompositeProducer(vectors); |
| 143 | + |
| 144 | + producer.produce(encoder); |
| 145 | + producer.produce(encoder); |
| 146 | + } |
| 147 | + |
| 148 | + List<Field> roundTripFields = new ArrayList<>(); |
| 149 | + List<FieldVector> roundTripVectors = new ArrayList<>(); |
| 150 | + |
| 151 | + try (FileInputStream fis = new FileInputStream(dataFile)) { |
| 152 | + |
| 153 | + BinaryDecoder decoder = new DecoderFactory().directBinaryDecoder(fis, null); |
| 154 | + CompositeAvroConsumer consumer = AvroToArrowUtils.createCompositeConsumer(schema, config); |
| 155 | + |
| 156 | + consumer.getConsumers().forEach(c -> roundTripFields.add(c.getVector().getField())); |
| 157 | + consumer.getConsumers().forEach(c -> roundTripVectors.add(c.getVector())); |
| 158 | + consumer.consume(decoder); |
| 159 | + consumer.consume(decoder); |
| 160 | + } |
| 161 | + |
| 162 | + VectorSchemaRoot root = new VectorSchemaRoot(fields, vectors, 2); |
| 163 | + VectorSchemaRoot roundTripRoot = new VectorSchemaRoot(roundTripFields, roundTripVectors, 2); |
| 164 | + |
| 165 | + assertEquals(root.getRowCount(), roundTripRoot.getRowCount()); |
| 166 | + |
| 167 | + for (int row = 0; row < 2; row++) { |
| 168 | + for (int col = 0; col < 3; col++) { |
| 169 | + FieldVector vector = root.getVector(col); |
| 170 | + FieldVector roundTripVector = roundTripRoot.getVector(col); |
| 171 | + assertEquals(vector.getObject(row), roundTripVector.getObject(row)); |
| 172 | + } |
| 173 | + } |
| 174 | + } |
85 | 175 | } |
0 commit comments