NIFI-9721: Support enum types in AvroTypeUtil.buildAvroSchema()

Signed-off-by: Joe Gresock <jgresock@gmail.com>

This closes #5793.
This commit is contained in:
Matthew Burgess 2022-02-23 11:06:41 -05:00 committed by Joe Gresock
parent 3511ce3d13
commit ab4cadc204
No known key found for this signature in database
GPG Key ID: 37F5B9B6E258C8B7
2 changed files with 16 additions and 0 deletions

View File

@ -46,6 +46,7 @@ import org.apache.nifi.serialization.record.StandardSchemaIdentifier;
import org.apache.nifi.serialization.record.type.ArrayDataType;
import org.apache.nifi.serialization.record.type.ChoiceDataType;
import org.apache.nifi.serialization.record.type.DecimalDataType;
import org.apache.nifi.serialization.record.type.EnumDataType;
import org.apache.nifi.serialization.record.type.MapDataType;
import org.apache.nifi.serialization.record.type.RecordDataType;
import org.apache.nifi.serialization.record.util.DataTypeUtils;
@ -298,6 +299,10 @@ public class AvroTypeUtil {
schema = Schema.create(Type.LONG);
LogicalTypes.timestampMillis().addToSchema(schema);
break;
case ENUM:
final EnumDataType enumType = (EnumDataType) dataType;
schema = Schema.createEnum(fieldName, "", "org.apache.nifi", enumType.getEnums());
break;
default:
return null;
}

View File

@ -1155,6 +1155,17 @@ public class TestAvroTypeUtil {
assertEquals(expected, actual.toString());
}
@Test
public void testBuildEnumSchema() {
final List<RecordField> fields = new ArrayList<>();
final DataType enumDataType = RecordFieldType.ENUM.getEnumDataType(Arrays.asList("YES", "NO"));
fields.add(new RecordField("enumField", enumDataType));
RecordSchema recordSchema = new SimpleRecordSchema(fields);
Schema actual = AvroTypeUtil.extractAvroSchema(recordSchema);
assertNotNull(actual);
}
private MapRecord givenRecordContainingNumericMap() {
final Map<String, Object> numberValues = new HashMap<>();