|
20 | 20 |
|
21 | 21 | import java.util.Arrays; |
22 | 22 | import java.util.List; |
| 23 | +import org.apache.arrow.memory.BufferAllocator; |
| 24 | +import org.apache.arrow.memory.RootAllocator; |
| 25 | +import org.apache.arrow.vector.VarCharVector; |
| 26 | +import org.apache.arrow.vector.dictionary.Dictionary; |
| 27 | +import org.apache.arrow.vector.dictionary.DictionaryProvider; |
23 | 28 | import org.apache.arrow.vector.types.DateUnit; |
24 | 29 | import org.apache.arrow.vector.types.FloatingPointPrecision; |
25 | 30 | import org.apache.arrow.vector.types.TimeUnit; |
26 | 31 | import org.apache.arrow.vector.types.UnionMode; |
27 | 32 | import org.apache.arrow.vector.types.pojo.ArrowType; |
| 33 | +import org.apache.arrow.vector.types.pojo.DictionaryEncoding; |
28 | 34 | import org.apache.arrow.vector.types.pojo.Field; |
29 | 35 | import org.apache.arrow.vector.types.pojo.FieldType; |
30 | 36 | import org.apache.avro.LogicalTypes; |
@@ -1389,4 +1395,46 @@ public void testConvertUnionTypes() { |
1389 | 1395 | Schema.Type.STRING, |
1390 | 1396 | schema.getField("nullableDenseUnionField").schema().getTypes().get(3).getType()); |
1391 | 1397 | } |
| 1398 | + |
| 1399 | + @Test |
| 1400 | + public void testWriteDictEnumEncoded() { |
| 1401 | + |
| 1402 | + BufferAllocator allocator = new RootAllocator(); |
| 1403 | + |
| 1404 | + // Create a dictionary |
| 1405 | + FieldType dictionaryField = new FieldType(false, new ArrowType.Utf8(), null); |
| 1406 | + VarCharVector dictionaryVector = |
| 1407 | + new VarCharVector(new Field("dictionary", dictionaryField, null), allocator); |
| 1408 | + |
| 1409 | + dictionaryVector.allocateNew(3); |
| 1410 | + dictionaryVector.set(0, "apple".getBytes()); |
| 1411 | + dictionaryVector.set(1, "banana".getBytes()); |
| 1412 | + dictionaryVector.set(2, "cherry".getBytes()); |
| 1413 | + dictionaryVector.setValueCount(3); |
| 1414 | + |
| 1415 | + Dictionary dictionary = |
| 1416 | + new Dictionary( |
| 1417 | + dictionaryVector, new DictionaryEncoding(0L, false, new ArrowType.Int(8, true))); |
| 1418 | + DictionaryProvider dictionaries = new DictionaryProvider.MapDictionaryProvider(dictionary); |
| 1419 | + |
| 1420 | + List<Field> fields = |
| 1421 | + Arrays.asList( |
| 1422 | + new Field( |
| 1423 | + "enumField", |
| 1424 | + new FieldType(false, new ArrowType.Int(8, true), dictionary.getEncoding(), null), |
| 1425 | + null)); |
| 1426 | + |
| 1427 | + Schema schema = ArrowToAvroUtils.createAvroSchema(fields, "TestRecord", null, dictionaries); |
| 1428 | + |
| 1429 | + assertEquals(Schema.Type.RECORD, schema.getType()); |
| 1430 | + assertEquals(1, schema.getFields().size()); |
| 1431 | + |
| 1432 | + Schema.Field enumField = schema.getField("enumField"); |
| 1433 | + |
| 1434 | + assertEquals(Schema.Type.ENUM, enumField.schema().getType()); |
| 1435 | + assertEquals(3, enumField.schema().getEnumSymbols().size()); |
| 1436 | + assertEquals("apple", enumField.schema().getEnumSymbols().get(0)); |
| 1437 | + assertEquals("banana", enumField.schema().getEnumSymbols().get(1)); |
| 1438 | + assertEquals("cherry", enumField.schema().getEnumSymbols().get(2)); |
| 1439 | + } |
1392 | 1440 | } |
0 commit comments