2424import org .apache .arrow .adapter .avro .producers .AvroBigIntProducer ;
2525import org .apache .arrow .adapter .avro .producers .AvroBooleanProducer ;
2626import org .apache .arrow .adapter .avro .producers .AvroBytesProducer ;
27+ import org .apache .arrow .adapter .avro .producers .AvroEnumProducer ;
2728import org .apache .arrow .adapter .avro .producers .AvroFixedSizeBinaryProducer ;
2829import org .apache .arrow .adapter .avro .producers .AvroFixedSizeListProducer ;
2930import org .apache .arrow .adapter .avro .producers .AvroFloat2Producer ;
6263import org .apache .arrow .adapter .avro .producers .logical .AvroTimestampSecProducer ;
6364import org .apache .arrow .adapter .avro .producers .logical .AvroTimestampSecTzProducer ;
6465import org .apache .arrow .util .Preconditions ;
66+ import org .apache .arrow .vector .BaseIntVector ;
6567import org .apache .arrow .vector .BigIntVector ;
6668import org .apache .arrow .vector .BitVector ;
6769import org .apache .arrow .vector .DateDayVector ;
100102import org .apache .arrow .vector .complex .MapVector ;
101103import org .apache .arrow .vector .complex .StructVector ;
102104import org .apache .arrow .vector .dictionary .Dictionary ;
103- import org .apache .arrow .vector .dictionary .DictionaryEncoder ;
104105import org .apache .arrow .vector .dictionary .DictionaryProvider ;
105106import org .apache .arrow .vector .types .FloatingPointPrecision ;
106107import org .apache .arrow .vector .types .TimeUnit ;
@@ -331,12 +332,8 @@ private static <T> T buildBaseTypeSchema(
331332 String [] symbols = dictionarySymbols (dictionary );
332333 return builder .enumeration (field .getName ()).symbols (symbols );
333334 } else {
334- Field decodedField =
335- new Field (
336- field .getName (),
337- dictionary .getVector ().getField ().getFieldType (),
338- dictionary .getVector ().getField ().getChildren ());
339- return buildBaseTypeSchema (builder , decodedField , namespace , dictionaries );
335+ throw new IllegalArgumentException (
336+ "Dictionary-encoded field is not a valid enum: " + field .getName ());
340337 }
341338 }
342339
@@ -449,7 +446,8 @@ private static LogicalType timestampLogicalType(ArrowType.Timestamp timestampTyp
449446
450447 private static boolean dictionaryIsValidEnum (Dictionary dictionary ) {
451448
452- if (dictionary .getVectorType ().getTypeID () != ArrowType .ArrowTypeID .Utf8 ) {
449+ if (dictionary .getVectorType ().getTypeID () != ArrowType .ArrowTypeID .Utf8
450+ || !(dictionary .getVector () instanceof VarCharVector )) {
453451 return false ;
454452 }
455453
@@ -502,33 +500,24 @@ private static String[] dictionarySymbols(Dictionary dictionary) {
502500 * @param vectors The vectors that will be used to produce Avro data
503501 * @return The resulting composite Avro producer
504502 */
505- public static CompositeAvroProducer createCompositeProducer (
506- List <FieldVector > vectors , DictionaryProvider dictionaries ) {
503+ public static CompositeAvroProducer createCompositeProducer (List <FieldVector > vectors ) {
507504
508505 List <Producer <? extends FieldVector >> producers = new ArrayList <>(vectors .size ());
509506
510507 for (FieldVector vector : vectors ) {
511- BaseAvroProducer <? extends FieldVector > producer = createProducer (vector , dictionaries );
508+ BaseAvroProducer <? extends FieldVector > producer = createProducer (vector );
512509 producers .add (producer );
513510 }
514511
515512 return new CompositeAvroProducer (producers );
516513 }
517514
518- /** Overload provided for convenience, sets dictionaries = null. */
519- public static CompositeAvroProducer createCompositeProducer (List <FieldVector > vectors ) {
520-
521- return createCompositeProducer (vectors , null );
522- }
523-
524- private static BaseAvroProducer <?> createProducer (
525- FieldVector vector , DictionaryProvider dictionaries ) {
515+ private static BaseAvroProducer <?> createProducer (FieldVector vector ) {
526516 boolean nullable = vector .getField ().isNullable ();
527- return createProducer (vector , nullable , dictionaries );
517+ return createProducer (vector , nullable );
528518 }
529519
530- private static BaseAvroProducer <?> createProducer (
531- FieldVector vector , boolean nullable , DictionaryProvider dictionaries ) {
520+ private static BaseAvroProducer <?> createProducer (FieldVector vector , boolean nullable ) {
532521
533522 Preconditions .checkNotNull (vector , "Arrow vector object can't be null" );
534523
@@ -537,30 +526,13 @@ private static BaseAvroProducer<?> createProducer(
537526 // Avro understands nullable types as a union of type | null
538527 // Most nullable fields in a VSR will not be unions, so provide a special wrapper
539528 if (nullable && minorType != Types .MinorType .UNION ) {
540- final BaseAvroProducer <?> innerProducer = createProducer (vector , false , dictionaries );
529+ final BaseAvroProducer <?> innerProducer = createProducer (vector , false );
541530 return new AvroNullableProducer <>(innerProducer );
542531 }
543532
544533 if (vector .getField ().getDictionary () != null ) {
545- if (dictionaries == null ) {
546- throw new IllegalArgumentException (
547- "Field references a dictionary but no dictionaries were provided: "
548- + vector .getField ().getName ());
549- }
550- Dictionary dictionary = dictionaries .lookup (vector .getField ().getDictionary ().getId ());
551- if (dictionary == null ) {
552- throw new IllegalArgumentException (
553- "Field references a dictionary that does not exist: "
554- + vector .getField ().getName ()
555- + ", dictionary ID = "
556- + vector .getField ().getDictionary ().getId ());
557- }
558- // If a field is dictionary-encoded but cannot be represented as an Avro enum,
559- // then decode it before writing
560- if (!dictionaryIsValidEnum (dictionary )) {
561- FieldVector decodedVector = (FieldVector ) DictionaryEncoder .decode (vector , dictionary );
562- return createProducer (decodedVector , nullable , dictionaries );
563- }
534+ BaseIntVector dictEncodedVector = (BaseIntVector ) vector ;
535+ return new AvroEnumProducer (dictEncodedVector );
564536 }
565537
566538 switch (minorType ) {
@@ -640,23 +612,21 @@ private static BaseAvroProducer<?> createProducer(
640612 Producer <?>[] childProducers = new Producer <?>[childVectors .size ()];
641613 for (int i = 0 ; i < childVectors .size (); i ++) {
642614 FieldVector childVector = childVectors .get (i );
643- childProducers [i ] =
644- createProducer (childVector , childVector .getField ().isNullable (), dictionaries );
615+ childProducers [i ] = createProducer (childVector , childVector .getField ().isNullable ());
645616 }
646617 return new AvroStructProducer (structVector , childProducers );
647618
648619 case LIST :
649620 ListVector listVector = (ListVector ) vector ;
650621 FieldVector itemVector = listVector .getDataVector ();
651- Producer <?> itemProducer =
652- createProducer (itemVector , itemVector .getField ().isNullable (), dictionaries );
622+ Producer <?> itemProducer = createProducer (itemVector , itemVector .getField ().isNullable ());
653623 return new AvroListProducer (listVector , itemProducer );
654624
655625 case FIXED_SIZE_LIST :
656626 FixedSizeListVector fixedListVector = (FixedSizeListVector ) vector ;
657627 FieldVector fixedItemVector = fixedListVector .getDataVector ();
658628 Producer <?> fixedItemProducer =
659- createProducer (fixedItemVector , fixedItemVector .getField ().isNullable (), dictionaries );
629+ createProducer (fixedItemVector , fixedItemVector .getField ().isNullable ());
660630 return new AvroFixedSizeListProducer (fixedListVector , fixedItemProducer );
661631
662632 case MAP :
@@ -670,7 +640,7 @@ private static BaseAvroProducer<?> createProducer(
670640 FieldVector valueVector = entryVector .getChildrenFromFields ().get (1 );
671641 Producer <?> keyProducer = new AvroStringProducer (keyVector );
672642 Producer <?> valueProducer =
673- createProducer (valueVector , valueVector .getField ().isNullable (), dictionaries );
643+ createProducer (valueVector , valueVector .getField ().isNullable ());
674644 Producer <?> entryProducer =
675645 new AvroStructProducer (entryVector , new Producer <?>[] {keyProducer , valueProducer });
676646 return new AvroMapProducer (mapVector , entryProducer );
0 commit comments