This is an automated email from the ASF dual-hosted git repository.

tpalfy pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 7d14d0baf7 NIFI-14646: Fix data type retrieval error on nested record 
types in Protobuf Reader
7d14d0baf7 is described below

commit 7d14d0baf76766b97b919f758fd729151388d336
Author: Mark Bathori <[email protected]>
AuthorDate: Fri Jun 13 13:06:04 2025 +0200

    NIFI-14646: Fix data type retrieval error on nested record types in 
Protobuf Reader
    
    This closes #10017.
    
    Signed-off-by: Tamas Palfy <[email protected]>
---
 .../protobuf/converter/ProtobufDataConverter.java  |  80 ++++++++++-----------
 .../nifi/services/protobuf/ProtoTestUtil.java      |  16 +++--
 .../converter/TestProtobufDataConverter.java       |  13 ++--
 .../protobuf/schema/TestProtoSchemaParser.java     |  13 ++--
 .../src/test/resources/test_proto3.desc            | Bin 984 -> 1060 bytes
 .../src/test/resources/test_proto3.proto           |  12 ++--
 6 files changed, 76 insertions(+), 58 deletions(-)

diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/converter/ProtobufDataConverter.java
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/converter/ProtobufDataConverter.java
index fa36441912..90a519b3f7 100644
--- 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/converter/ProtobufDataConverter.java
+++ 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/main/java/org/apache/nifi/services/protobuf/converter/ProtobufDataConverter.java
@@ -111,10 +111,10 @@ public class ProtobufDataConverter {
 
         if ((ANY_MESSAGE_TYPE).equals(messageType.getType().toString())) {
             containsAnyField = true;
-            return handleAnyField(unknownFieldSet);
+            return handleAnyField(unknownFieldSet, recordSchema);
         }
 
-        final Map<String, Object> fieldValues = 
processMessageFields(messageType, unknownFieldSet);
+        final Map<String, Object> fieldValues = 
processMessageFields(messageType, unknownFieldSet, recordSchema);
         return new MapRecord(recordSchema, fieldValues, false, 
dropUnknownFields);
     }
 
@@ -125,20 +125,20 @@ public class ProtobufDataConverter {
      * @param unknownFieldSet received proto data fields
      * @return Map of processed fields
      */
-    private Map<String, Object> processMessageFields(MessageType messageType, 
UnknownFieldSet unknownFieldSet) throws InvalidProtocolBufferException {
+    private Map<String, Object> processMessageFields(MessageType messageType, 
UnknownFieldSet unknownFieldSet, RecordSchema recordSchema) throws 
InvalidProtocolBufferException {
         Map<String, Object> recordValues = new HashMap<>();
 
         for (final Field field : messageType.getDeclaredFields()) {
-            collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()));
+            collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordSchema);
         }
 
         for (final Field field : messageType.getExtensionFields()) {
-            collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()));
+            collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordSchema);
         }
 
         for (final OneOf oneOf : messageType.getOneOfs()) {
             for (Field field : oneOf.getFields()) {
-                collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()));
+                collectFieldValue(recordValues, new ProtoField(field), 
unknownFieldSet.getField(field.getTag()), recordSchema);
             }
         }
         return recordValues;
@@ -151,33 +151,34 @@ public class ProtobufDataConverter {
      * @param protoField                proto field's properties
      * @param unknownField              field's value
      */
-    private void collectFieldValue(Map<String, Object> 
fieldNameToConvertedValue, ProtoField protoField, UnknownFieldSet.Field 
unknownField) throws InvalidProtocolBufferException {
-        final Optional<Object> fieldValue = convertFieldValues(protoField, 
unknownField);
+    private void collectFieldValue(Map<String, Object> 
fieldNameToConvertedValue, ProtoField protoField,
+                                   UnknownFieldSet.Field unknownField, 
RecordSchema recordSchema) throws InvalidProtocolBufferException {
+        final Optional<Object> fieldValue = convertFieldValues(protoField, 
unknownField, recordSchema);
         fieldValue.ifPresent(value -> 
fieldNameToConvertedValue.put(protoField.getFieldName(), value));
     }
 
-    private Optional<Object> convertFieldValues(ProtoField protoField, 
UnknownFieldSet.Field unknownField) throws InvalidProtocolBufferException {
+    private Optional<Object> convertFieldValues(ProtoField protoField, 
UnknownFieldSet.Field unknownField, RecordSchema recordSchema) throws 
InvalidProtocolBufferException {
         if (!unknownField.getLengthDelimitedList().isEmpty()) {
             if (protoField.isRepeatable() && 
!isLengthDelimitedType(protoField)) {
-                return Optional.of(convertRepeatedFields(protoField, 
unknownField.getLengthDelimitedList()));
+                return Optional.of(convertRepeatedFields(protoField, 
unknownField.getLengthDelimitedList(), recordSchema));
             } else {
-                return Optional.of(convertLengthDelimitedFields(protoField, 
unknownField.getLengthDelimitedList()));
+                return Optional.of(convertLengthDelimitedFields(protoField, 
unknownField.getLengthDelimitedList(), recordSchema));
             }
         }
         if (!unknownField.getFixed32List().isEmpty()) {
-            return Optional.of(convertFixed32Fields(protoField, 
unknownField.getFixed32List()));
+            return Optional.of(convertFixed32Fields(protoField, 
unknownField.getFixed32List(), recordSchema));
         }
         if (!unknownField.getFixed64List().isEmpty()) {
-            return Optional.of(convertFixed64Fields(protoField, 
unknownField.getFixed64List()));
+            return Optional.of(convertFixed64Fields(protoField, 
unknownField.getFixed64List(), recordSchema));
         }
         if (!unknownField.getVarintList().isEmpty()) {
-            return Optional.of(convertVarintFields(protoField, 
unknownField.getVarintList()));
+            return Optional.of(convertVarintFields(protoField, 
unknownField.getVarintList(), recordSchema));
         }
 
         return Optional.empty();
     }
 
-    private Object convertRepeatedFields(ProtoField protoField, 
List<ByteString> fieldValues) {
+    private Object convertRepeatedFields(ProtoField protoField, 
List<ByteString> fieldValues, RecordSchema recordSchema) {
         final CodedInputStream inputStream = 
fieldValues.getFirst().newCodedInput();
         final ProtoType protoType = protoField.getProtoType();
         if (protoType.isScalar()) {
@@ -198,10 +199,10 @@ public class ProtobufDataConverter {
                 default -> throw new 
IllegalStateException(String.format("Unexpected type [%s] was received for 
field [%s]",
                         protoType.getSimpleName(), protoField.getFieldName()));
             };
-            return resolveFieldValue(protoField, 
processRepeatedValues(inputStream, valueReader), value -> value);
+            return resolveFieldValue(protoField, 
processRepeatedValues(inputStream, valueReader), value -> value, recordSchema);
         } else {
             List<Integer> values = processRepeatedValues(inputStream, 
CodedInputStream::readEnum);
-            return resolveFieldValue(protoField, values, value -> 
convertEnum(value, protoType));
+            return resolveFieldValue(protoField, values, value -> 
convertEnum(value, protoType), recordSchema);
         }
     }
 
@@ -213,7 +214,7 @@ public class ProtobufDataConverter {
      * @return converted field values
      * @throws InvalidProtocolBufferException failed to parse input data
      */
-    private Object convertLengthDelimitedFields(ProtoField protoField, 
List<ByteString> values) throws InvalidProtocolBufferException {
+    private Object convertLengthDelimitedFields(ProtoField protoField, 
List<ByteString> values, RecordSchema recordSchema) throws 
InvalidProtocolBufferException {
         final ProtoType protoType = protoField.getProtoType();
         final Function<ByteString, Object> valueConverter;
         if (protoType.isScalar()) {
@@ -225,28 +226,27 @@ public class ProtobufDataConverter {
                                 " [%s] is not LengthDelimited field type", 
protoField.getFieldName(), protoType.getSimpleName()));
             };
         } else if (protoType.isMap()) {
-            return createMap(protoType, values);
+            return createMap(protoType, values, recordSchema);
         } else {
             final MessageType messageType = (MessageType) 
schema.getType(protoType);
             Objects.requireNonNull(messageType, String.format("Message type 
with name [%s] not found in the provided proto files", protoType));
 
             valueConverter = value -> {
                 try {
-                    Optional<DataType> recordDataType = 
rootRecordSchema.getDataType(protoField.getFieldName());
+                    Optional<DataType> recordDataType = 
recordSchema.getDataType(protoField.getFieldName());
                     if (protoField.isRepeatable()) {
-                        final ArrayDataType arrayDataType = (ArrayDataType) 
recordDataType.get();
-                        recordDataType = 
Optional.ofNullable(arrayDataType.getElementType());
+                        recordDataType = recordDataType.map(dataType -> 
((ArrayDataType) dataType).getElementType());
                     }
-                    RecordSchema recordSchema = recordDataType.map(dataType ->
+                    final RecordSchema subSchema = recordDataType.map(dataType 
->
                             ((RecordDataType) 
dataType).getChildSchema()).orElse(generateRecordSchema(messageType.getType().toString()));
-                    return createRecord(messageType, value, recordSchema);
+                    return createRecord(messageType, value, subSchema);
                 } catch (InvalidProtocolBufferException e) {
                     throw new IllegalStateException("Failed to create record 
from the provided input data for field " + protoField.getFieldName(), e);
                 }
             };
         }
 
-        return resolveFieldValue(protoField, values, valueConverter);
+        return resolveFieldValue(protoField, values, valueConverter, 
recordSchema);
     }
 
     /**
@@ -256,7 +256,7 @@ public class ProtobufDataConverter {
      * @param values     field's unprocessed values
      * @return converted field values
      */
-    private Object convertFixed32Fields(ProtoField protoField, List<Integer> 
values) {
+    private Object convertFixed32Fields(ProtoField protoField, List<Integer> 
values, RecordSchema recordSchema) {
         final String typeName = protoField.getProtoType().getSimpleName();
         final Function<Integer, Object> valueConverter =
                 switch (FieldType.findValue(typeName)) {
@@ -268,7 +268,7 @@ public class ProtobufDataConverter {
                                     " [%s] is not Fixed32 field type", 
protoField.getFieldName(), typeName));
                 };
 
-        return resolveFieldValue(protoField, values, valueConverter);
+        return resolveFieldValue(protoField, values, valueConverter, 
recordSchema);
     }
 
     /**
@@ -278,7 +278,7 @@ public class ProtobufDataConverter {
      * @param values     field's unprocessed values
      * @return converted field values
      */
-    private Object convertFixed64Fields(ProtoField protoField, List<Long> 
values) {
+    private Object convertFixed64Fields(ProtoField protoField, List<Long> 
values, RecordSchema recordSchema) {
         final String typeName = protoField.getProtoType().getSimpleName();
         final Function<Long, Object> valueConverter =
                 switch (FieldType.findValue(typeName)) {
@@ -290,7 +290,7 @@ public class ProtobufDataConverter {
                                     " [%s] is not Fixed64 field type", 
protoField.getFieldName(), typeName));
                 };
 
-        return resolveFieldValue(protoField, values, valueConverter);
+        return resolveFieldValue(protoField, values, valueConverter, 
recordSchema);
     }
 
     /**
@@ -300,7 +300,7 @@ public class ProtobufDataConverter {
      * @param values     field's unprocessed values
      * @return converted field values
      */
-    private Object convertVarintFields(ProtoField protoField, List<Long> 
values) {
+    private Object convertVarintFields(ProtoField protoField, List<Long> 
values, RecordSchema recordSchema) {
         final ProtoType protoType = protoField.getProtoType();
         final Function<Long, Object> valueConverter;
         if (protoType.isScalar()) {
@@ -319,14 +319,14 @@ public class ProtobufDataConverter {
             valueConverter = value -> convertEnum(value.intValue(), protoType);
         }
 
-        return resolveFieldValue(protoField, values, valueConverter);
+        return resolveFieldValue(protoField, values, valueConverter, 
recordSchema);
     }
 
-    private <T> Object resolveFieldValue(ProtoField protoField, List<T> 
values, Function<T, Object> valueConverter) {
+    private <T> Object resolveFieldValue(ProtoField protoField, List<T> 
values, Function<T, Object> valueConverter, RecordSchema recordSchema) {
         List<Object> resultValues = 
values.stream().map(valueConverter).toList();
 
         if (coerceTypes) {
-            final Optional<RecordField> recordField = 
rootRecordSchema.getField(protoField.getFieldName());
+            final Optional<RecordField> recordField = 
recordSchema.getField(protoField.getFieldName());
             if (recordField.isPresent()) {
                 final DataType dataType;
                 if (protoField.isRepeatable()) {
@@ -354,15 +354,15 @@ public class ProtobufDataConverter {
      * @return created Map
      * @throws InvalidProtocolBufferException failed to parse input data
      */
-    private Map<String, Object> createMap(ProtoType protoType, 
List<ByteString> data) throws InvalidProtocolBufferException {
+    private Map<String, Object> createMap(ProtoType protoType, 
List<ByteString> data, RecordSchema recordSchema) throws 
InvalidProtocolBufferException {
         Map<String, Object> mapResult = new HashMap<>();
 
         for (final ByteString entry : data) {
             final UnknownFieldSet unknownFieldSet = 
UnknownFieldSet.parseFrom(entry);
             Map<String, Object> mapEntry = new HashMap<>();
 
-            collectFieldValue(mapEntry, new ProtoField(MAP_KEY_FIELD_NAME, 
protoType.getKeyType()), unknownFieldSet.getField(1));
-            collectFieldValue(mapEntry, new ProtoField(MAP_VALUE_FIELD_NAME, 
protoType.getValueType()), unknownFieldSet.getField(2));
+            collectFieldValue(mapEntry, new ProtoField(MAP_KEY_FIELD_NAME, 
protoType.getKeyType()), unknownFieldSet.getField(1), recordSchema);
+            collectFieldValue(mapEntry, new ProtoField(MAP_VALUE_FIELD_NAME, 
protoType.getValueType()), unknownFieldSet.getField(2), recordSchema);
 
             mapResult.put(String.valueOf(mapEntry.get(MAP_KEY_FIELD_NAME)), 
mapEntry.get(MAP_VALUE_FIELD_NAME));
         }
@@ -384,17 +384,17 @@ public class ProtobufDataConverter {
      * @return created record from the parsed message
      * @throws InvalidProtocolBufferException failed to parse input data
      */
-    private MapRecord handleAnyField(UnknownFieldSet unknownFieldSet) throws 
InvalidProtocolBufferException {
+    private MapRecord handleAnyField(UnknownFieldSet unknownFieldSet, 
RecordSchema recordSchema) throws InvalidProtocolBufferException {
         Map<String, Object> recordValues = new HashMap<>();
-        collectFieldValue(recordValues, new 
ProtoField(ANY_TYPE_URL_FIELD_NAME, ProtoType.STRING), 
unknownFieldSet.getField(1));
-        collectFieldValue(recordValues, new ProtoField(ANY_VALUE_FIELD_NAME, 
ProtoType.BYTES), unknownFieldSet.getField(2));
+        collectFieldValue(recordValues, new 
ProtoField(ANY_TYPE_URL_FIELD_NAME, ProtoType.STRING), 
unknownFieldSet.getField(1), recordSchema);
+        collectFieldValue(recordValues, new ProtoField(ANY_VALUE_FIELD_NAME, 
ProtoType.BYTES), unknownFieldSet.getField(2), recordSchema);
 
         final String typeName = 
String.valueOf(recordValues.get(ANY_TYPE_URL_FIELD_NAME));
         final UnknownFieldSet anyFieldSet = UnknownFieldSet.parseFrom((byte[]) 
recordValues.get(ANY_VALUE_FIELD_NAME));
         final MessageType messageType = (MessageType) 
schema.getType(getQualifiedTypeName(typeName));
         Objects.requireNonNull(messageType, String.format("Message type with 
name [%s] not found in the provided proto files", typeName));
 
-        return new MapRecord(generateRecordSchema(typeName), 
processMessageFields(messageType, anyFieldSet), false, dropUnknownFields);
+        return new MapRecord(generateRecordSchema(typeName), 
processMessageFields(messageType, anyFieldSet, recordSchema), false, 
dropUnknownFields);
     }
 
     /**
diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/ProtoTestUtil.java
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/ProtoTestUtil.java
index 2d0b8bd3b8..3293513b82 100644
--- 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/ProtoTestUtil.java
+++ 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/ProtoTestUtil.java
@@ -70,8 +70,9 @@ public class ProtoTestUtil {
 
         Descriptors.Descriptor messageDescriptor = 
fileDescriptor.findMessageTypeByName("Proto3Message");
         Descriptors.Descriptor nestedMessageDescriptor = 
fileDescriptor.findMessageTypeByName("NestedMessage");
+        Descriptors.Descriptor nestedMessageDescriptor2 = 
fileDescriptor.findMessageTypeByName("NestedMessage2");
         Descriptors.EnumDescriptor enumValueDescriptor = 
fileDescriptor.findEnumTypeByName("TestEnum");
-        Descriptors.Descriptor mapDescriptor = 
nestedMessageDescriptor.findNestedTypeByName("TestMapEntry");
+        Descriptors.Descriptor mapDescriptor = 
nestedMessageDescriptor2.findNestedTypeByName("TestMapEntry");
 
         DynamicMessage mapEntry1 = DynamicMessage
                 .newBuilder(mapDescriptor)
@@ -85,13 +86,18 @@ public class ProtoTestUtil {
                 .setField(mapDescriptor.findFieldByName(MAP_VALUE_FIELD_NAME), 
202)
                 .build();
 
+        DynamicMessage nestedMessage2 = DynamicMessage
+                .newBuilder(nestedMessageDescriptor2)
+                .setField(nestedMessageDescriptor2.findFieldByNumber(30), 
Arrays.asList(mapEntry1, mapEntry2))
+                .setField(nestedMessageDescriptor2.findFieldByNumber(31), "One 
Of Option")
+                .setField(nestedMessageDescriptor2.findFieldByNumber(32), true)
+                .setField(nestedMessageDescriptor2.findFieldByNumber(33), 3)
+                .build();
+
         DynamicMessage nestedMessage = DynamicMessage
                 .newBuilder(nestedMessageDescriptor)
                 .setField(nestedMessageDescriptor.findFieldByNumber(20), 
enumValueDescriptor.findValueByNumber(2))
-                .setField(nestedMessageDescriptor.findFieldByNumber(21), 
Arrays.asList(mapEntry1, mapEntry2))
-                .setField(nestedMessageDescriptor.findFieldByNumber(22), "One 
Of Option")
-                .setField(nestedMessageDescriptor.findFieldByNumber(23), true)
-                .setField(nestedMessageDescriptor.findFieldByNumber(24), 3)
+                
.addRepeatedField(nestedMessageDescriptor.findFieldByNumber(21), nestedMessage2)
                 .build();
 
         DynamicMessage message = DynamicMessage
diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/converter/TestProtobufDataConverter.java
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/converter/TestProtobufDataConverter.java
index 7aeafa895a..8690d6b7eb 100644
--- 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/converter/TestProtobufDataConverter.java
+++ 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/converter/TestProtobufDataConverter.java
@@ -66,12 +66,17 @@ public class TestProtobufDataConverter {
         final MapRecord nestedRecord = (MapRecord) 
record.getValue("nestedMessage");
         assertEquals("ENUM_VALUE_3", nestedRecord.getValue("testEnum"));
 
-        assertEquals(Map.of("test_key_entry1", 101, "test_key_entry2", 202), 
nestedRecord.getValue("testMap"));
+        final Object[] recordList = (Object[]) 
nestedRecord.getValue("nestedMessage2");
+        assertEquals(1, recordList.length);
+
+        final MapRecord nestedRecord2 = (MapRecord) recordList[0];
+
+        assertEquals(Map.of("test_key_entry1", 101, "test_key_entry2", 202), 
nestedRecord2.getValue("testMap"));
 
         // assert only one field is set in the OneOf field
-        assertNull(nestedRecord.getValue("stringOption"));
-        assertNull(nestedRecord.getValue("booleanOption"));
-        assertEquals(3, nestedRecord.getValue("int32Option"));
+        assertNull(nestedRecord2.getValue("stringOption"));
+        assertNull(nestedRecord2.getValue("booleanOption"));
+        assertEquals(3, nestedRecord2.getValue("int32Option"));
     }
 
     @Test
diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/schema/TestProtoSchemaParser.java
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/schema/TestProtoSchemaParser.java
index 14009576fe..ceb012f161 100644
--- 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/schema/TestProtoSchemaParser.java
+++ 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/java/org/apache/nifi/services/protobuf/schema/TestProtoSchemaParser.java
@@ -58,11 +58,14 @@ public class TestProtoSchemaParser {
                 new RecordField("sfixed64Field", 
RecordFieldType.LONG.getDataType()),
                 new RecordField("nestedMessage", 
RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(Arrays.asList(
                         new RecordField("testEnum", 
RecordFieldType.ENUM.getEnumDataType(Arrays.asList("ENUM_VALUE_1", 
"ENUM_VALUE_2", "ENUM_VALUE_3"))),
-                        new RecordField("testMap", 
RecordFieldType.MAP.getMapDataType(RecordFieldType.INT.getDataType())),
-                        new RecordField("stringOption", 
RecordFieldType.STRING.getDataType()),
-                        new RecordField("booleanOption", 
RecordFieldType.BOOLEAN.getDataType()),
-                        new RecordField("int32Option", 
RecordFieldType.INT.getDataType())
-                ))))
+                        new RecordField("nestedMessage2", 
RecordFieldType.ARRAY.getArrayDataType(
+                                RecordFieldType.RECORD.getRecordDataType(new 
SimpleRecordSchema(Arrays.asList(
+                                        new RecordField("testMap", 
RecordFieldType.MAP.getMapDataType(RecordFieldType.INT.getDataType())),
+                                        new RecordField("stringOption", 
RecordFieldType.STRING.getDataType()),
+                                        new RecordField("booleanOption", 
RecordFieldType.BOOLEAN.getDataType()),
+                                        new RecordField("int32Option", 
RecordFieldType.INT.getDataType())
+                                )))
+                        ))))))
         ));
 
         final RecordSchema actual = schemaParser.createSchema("Proto3Message");
diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.desc
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.desc
index 1dbfb60613..985855dc32 100644
Binary files 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.desc
 and 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.desc
 differ
diff --git 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.proto
 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.proto
index 3e7a736cd3..522f8833ca 100644
--- 
a/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.proto
+++ 
b/nifi-extension-bundles/nifi-protobuf-bundle/nifi-protobuf-services/src/test/resources/test_proto3.proto
@@ -37,11 +37,15 @@ message Proto3Message {
 
 message NestedMessage {
   TestEnum testEnum = 20;
-  map<string, int32> testMap = 21;
+  repeated NestedMessage2 nestedMessage2 = 21;
+}
+
+message NestedMessage2 {
+  map<string, int32> testMap = 30;
   oneof oneOfField {
-    string stringOption = 22;
-    bool booleanOption = 23;
-    int32 int32Option = 24;
+    string stringOption = 31;
+    bool booleanOption = 32;
+    int32 int32Option = 33;
   }
 }
 

Reply via email to