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

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


The following commit(s) were added to refs/heads/master by this push:
     new 66b175f  NIFI-7390 Covering Avro type conversion in case of map 
withing Record
66b175f is described below

commit 66b175f405e727aba650f67872292ff8c8ccb14d
Author: Bence Simon <simonbence....@gmail.com>
AuthorDate: Wed May 6 16:04:36 2020 +0200

    NIFI-7390 Covering Avro type conversion in case of map withing Record
    
    Signed-off-by: Matthew Burgess <mattyb...@apache.org>
    
    This closes #4256
---
 .../java/org/apache/nifi/avro/AvroTypeUtil.java    |  2 +-
 .../org/apache/nifi/avro/TestAvroTypeUtil.java     | 51 ++++++++++++++++++++++
 2 files changed, 52 insertions(+), 1 deletion(-)

diff --git 
a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
 
b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
index caa2743..950e4cb 100644
--- 
a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
+++ 
b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java
@@ -747,7 +747,7 @@ public class AvroTypeUtil {
                     for (final RecordField recordField : 
recordValue.getSchema().getFields()) {
                         final Object v = recordValue.getValue(recordField);
                         if (v != null) {
-                            map.put(recordField.getFieldName(), v);
+                            map.put(recordField.getFieldName(), 
convertToAvroObject(v, fieldSchema.getValueType(), fieldName + "[" + 
recordField.getFieldName() + "]", charset));
                         }
                     }
 
diff --git 
a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
 
b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
index 5b5f55dd..412d573 100644
--- 
a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
+++ 
b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java
@@ -33,6 +33,7 @@ import org.apache.avro.util.Utf8;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.SimpleRecordSchema;
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.MapRecord;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
@@ -691,4 +692,54 @@ public class TestAvroTypeUtil {
         // THEN
         assertEquals(expected, actual);
     }
+
+    @Test
+    public void testConvertNifiRecordIntoAvroRecord() throws IOException {
+        // given
+        final MapRecord nifiRecord = givenRecordContainingNumericMap();
+        final Schema avroSchema = givenAvroSchemaContainingNumericMap();
+
+        // when
+        final GenericRecord result = AvroTypeUtil.createAvroRecord(nifiRecord, 
avroSchema);
+
+        // then
+        final HashMap<String, Object> numbers = (HashMap<String, Object>) 
result.get("numbers");
+        Assert.assertTrue(Long.class.isInstance(numbers.get("number1")));
+        Assert.assertTrue(Long.class.isInstance(numbers.get("number2")));
+    }
+
+    private MapRecord givenRecordContainingNumericMap() {
+
+        final Map<String, Object> numberValues = new HashMap<>();
+        numberValues.put("number1", 123); // Intentionally an Integer as 
validation accepts it
+        numberValues.put("number2", 123L);
+
+        final List<RecordField> numberFields = Arrays.asList(
+            new RecordField("number1", RecordFieldType.LONG.getDataType()),
+            new RecordField("number2", RecordFieldType.LONG.getDataType())
+        );
+
+        final RecordSchema nifiNumberSchema = new 
SimpleRecordSchema(numberFields);
+        final MapRecord numberRecord = new MapRecord(new 
SimpleRecordSchema(numberFields), numberValues);
+
+        final Map<String, Object> values = new HashMap<>();
+        values.put("id", 1);
+        values.put("numbers", numberRecord);
+
+        final List<RecordField> fields = Arrays.asList(
+                new RecordField("id", RecordFieldType.INT.getDataType()),
+                new RecordField("numbers", 
RecordFieldType.RECORD.getRecordDataType(nifiNumberSchema))
+        );
+
+        return new MapRecord(new SimpleRecordSchema(fields), values);
+    }
+
+    private Schema givenAvroSchemaContainingNumericMap() {
+        final List<Field> avroFields = Arrays.asList(
+                new Field("id", Schema.create(Type.INT), "", ""),
+                new Field("numbers", 
Schema.createMap(Schema.create(Type.LONG)), "", "")
+        );
+
+        return Schema.createRecord(avroFields);
+    }
 }

Reply via email to