You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ma...@apache.org on 2020/10/21 13:17:45 UTC

[nifi] branch main updated: NIFI-7925: ValidateRecord reports false positive for avro arrays with null elements

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

markap14 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 d05d0c6  NIFI-7925: ValidateRecord reports false positive for avro arrays with null elements
d05d0c6 is described below

commit d05d0c6240217da9cc72c07110ed273f37b0fa21
Author: Denes Arvay <de...@apache.org>
AuthorDate: Sat Oct 17 21:31:37 2020 +0200

    NIFI-7925: ValidateRecord reports false positive for avro arrays with null elements
---
 flow.xml.gz                                        | Bin 0 -> 2050 bytes
 .../nifi/serialization/record/RecordFieldType.java |  30 +++++++++++++++--
 .../serialization/record/type/ArrayDataType.java   |  37 ++++++++++++---------
 .../serialization/record/type/MapDataType.java     |  37 ++++++++++++---------
 .../java/org/apache/nifi/avro/AvroTypeUtil.java    |   7 ++--
 .../schema/validation/StandardSchemaValidator.java |  18 +++++++---
 .../processors/standard/TestValidateRecord.java    |  34 +++++++++++++++++++
 .../array-and-map-with-null-element.avro           | Bin 0 -> 281 bytes
 8 files changed, 125 insertions(+), 38 deletions(-)

diff --git a/flow.xml.gz b/flow.xml.gz
new file mode 100644
index 0000000..5f4d61a
Binary files /dev/null and b/flow.xml.gz differ
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
index 4e18fb9..7386882 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java
@@ -288,17 +288,30 @@ public enum RecordFieldType {
 
     /**
      * Returns a Data Type that represents an "ARRAY" type with the given element type.
+     * The returned array data type can't contain null elements.
      *
      * @param elementType the type of the arrays in the element
      * @return a DataType that represents an Array with the given element type, or <code>null</code> if this RecordFieldType
      *         is not the ARRAY type.
      */
     public DataType getArrayDataType(final DataType elementType) {
+        return getArrayDataType(elementType, ArrayDataType.DEFAULT_NULLABLE);
+    }
+
+    /**
+     * Returns a Data Type that represents an "ARRAY" type with the given element type.
+     *
+     * @param elementType the type of the arrays in the element
+     * @param elementsNullable indicates whether the array can contain null elements
+     * @return a DataType that represents an Array with the given element type, or <code>null</code> if this RecordFieldType
+     *         is not the ARRAY type.
+     */
+    public DataType getArrayDataType(final DataType elementType, final boolean elementsNullable) {
         if (this != ARRAY) {
             return null;
         }
 
-        return new ArrayDataType(elementType);
+        return new ArrayDataType(elementType, elementsNullable);
     }
 
 
@@ -341,17 +354,30 @@ public enum RecordFieldType {
 
     /**
      * Returns a Data Type that represents a "MAP" type with the given value type.
+     * The returned map data type can't contain null values.
      *
      * @param valueDataType the type of the values in the map
      * @return a DataType that represents a Map with the given value type, or <code>null</code> if this RecordFieldType
      *         is not the MAP type.
      */
     public DataType getMapDataType(final DataType valueDataType) {
+        return getMapDataType(valueDataType, MapDataType.DEFAULT_NULLABLE);
+    }
+
+    /**
+     * Returns a Data Type that represents a "MAP" type with the given value type.
+     *
+     * @param valueDataType the type of the values in the map
+     * @param valuesNullable indicates whether the map can contain null values
+     * @return a DataType that represents a Map with the given value type, or <code>null</code> if this RecordFieldType
+     *         is not the MAP type.
+     */
+    public DataType getMapDataType(final DataType valueDataType, boolean valuesNullable) {
         if (this != MAP) {
             return null;
         }
 
-        return new MapDataType(valueDataType);
+        return new MapDataType(valueDataType, valuesNullable);
     }
 
     /**
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
index f9f2569..26f4916 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
@@ -23,11 +23,20 @@ import org.apache.nifi.serialization.record.RecordFieldType;
 import java.util.Objects;
 
 public class ArrayDataType extends DataType {
+
+    public static final boolean DEFAULT_NULLABLE = false;
+
     private final DataType elementType;
+    private final boolean elementsNullable;
 
     public ArrayDataType(final DataType elementType) {
+        this(elementType, DEFAULT_NULLABLE);
+    }
+
+    public ArrayDataType(final DataType elementType, boolean elementsNullable) {
         super(RecordFieldType.ARRAY, null);
         this.elementType = elementType;
+        this.elementsNullable = elementsNullable;
     }
 
     public DataType getElementType() {
@@ -39,25 +48,23 @@ public class ArrayDataType extends DataType {
         return RecordFieldType.ARRAY;
     }
 
-    @Override
-    public int hashCode() {
-        return 31 + 41 * getFieldType().hashCode() + 41 * (elementType == null ? 0 : elementType.hashCode());
+    public boolean isElementsNullable() {
+        return elementsNullable;
     }
 
     @Override
-    public boolean equals(final Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-        if (!(obj instanceof ArrayDataType)) {
-            return false;
-        }
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof ArrayDataType)) return false;
+        if (!super.equals(o)) return false;
+        ArrayDataType that = (ArrayDataType) o;
+        return isElementsNullable() == that.isElementsNullable()
+                && Objects.equals(getElementType(), that.getElementType());
+    }
 
-        final ArrayDataType other = (ArrayDataType) obj;
-        return Objects.equals(elementType, other.elementType);
+    @Override
+    public int hashCode() {
+        return Objects.hash(super.hashCode(), getElementType(), isElementsNullable());
     }
 
     @Override
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
index 6435195..74c0683 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
@@ -23,41 +23,48 @@ import org.apache.nifi.serialization.record.RecordFieldType;
 import java.util.Objects;
 
 public class MapDataType extends DataType {
+
+    public static final boolean DEFAULT_NULLABLE = false;
+
     private final DataType valueType;
+    private final boolean valuesNullable;
 
     public MapDataType(final DataType elementType) {
+        this(elementType, DEFAULT_NULLABLE);
+    }
+
+    public MapDataType(final DataType elementType, boolean valuesNullable) {
         super(RecordFieldType.MAP, null);
         this.valueType = elementType;
+        this.valuesNullable = valuesNullable;
     }
 
     public DataType getValueType() {
         return valueType;
     }
 
+    public boolean isValuesNullable() {
+        return valuesNullable;
+    }
+
     @Override
     public RecordFieldType getFieldType() {
         return RecordFieldType.MAP;
     }
 
     @Override
-    public int hashCode() {
-        return 31 + 41 * getFieldType().hashCode() + 41 * (valueType == null ? 0 : valueType.hashCode());
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (!(o instanceof MapDataType)) return false;
+        if (!super.equals(o)) return false;
+        MapDataType that = (MapDataType) o;
+        return valuesNullable == that.valuesNullable
+                && Objects.equals(getValueType(), that.getValueType());
     }
 
     @Override
-    public boolean equals(final Object obj) {
-        if (obj == this) {
-            return true;
-        }
-        if (obj == null) {
-            return false;
-        }
-        if (!(obj instanceof MapDataType)) {
-            return false;
-        }
-
-        final MapDataType other = (MapDataType) obj;
-        return Objects.equals(valueType, other.valueType);
+    public int hashCode() {
+        return Objects.hash(super.hashCode(), getValueType(), valuesNullable);
     }
 
     @Override
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 3919efa..6d3157c 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
@@ -354,7 +354,9 @@ public class AvroTypeUtil {
 
         switch (avroType) {
             case ARRAY:
-                return RecordFieldType.ARRAY.getArrayDataType(determineDataType(avroSchema.getElementType(), knownRecordTypes));
+                final DataType elementType = determineDataType(avroSchema.getElementType(), knownRecordTypes);
+                final boolean elementsNullable = isNullable(avroSchema.getElementType());
+                return RecordFieldType.ARRAY.getArrayDataType(elementType, elementsNullable);
             case BYTES:
             case FIXED:
                 return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType());
@@ -403,7 +405,8 @@ public class AvroTypeUtil {
             case MAP:
                 final Schema valueSchema = avroSchema.getValueType();
                 final DataType valueType = determineDataType(valueSchema, knownRecordTypes);
-                return RecordFieldType.MAP.getMapDataType(valueType);
+                final boolean valuesNullable = isNullable(valueSchema);
+                return RecordFieldType.MAP.getMapDataType(valueType, valuesNullable);
             case UNION: {
                 final List<Schema> nonNullSubSchemas = getNonNullSubSchemas(avroSchema);
 
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java
index 18b7c7a..0f5384b 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java
@@ -69,7 +69,7 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
             if (validationContext.isStrictTypeChecking()) {
                 if (!isTypeCorrect(rawValue, dataType)) {
                     result.addValidationError(new StandardValidationError(concat(fieldPrefix, field), rawValue, ValidationErrorType.INVALID_FIELD,
-                        "Value is of type " + rawValue.getClass().getName() + " but was expected to be of type " + dataType));
+                        "Value is of type " + classNameOrNull(rawValue) + " but was expected to be of type " + dataType));
 
                     continue;
                 }
@@ -78,7 +78,7 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
                 // but will be false if the value is "123" and should be an Array or Record.
                 if (!DataTypeUtils.isCompatibleDataType(rawValue, dataType)) {
                     result.addValidationError(new StandardValidationError(concat(fieldPrefix, field), rawValue, ValidationErrorType.INVALID_FIELD,
-                        "Value is of type " + rawValue.getClass().getName() + " but was expected to be of type " + dataType));
+                        "Value is of type " + classNameOrNull(rawValue) + " but was expected to be of type " + dataType));
 
                     continue;
                 }
@@ -140,7 +140,7 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
 
             if (canonicalDataType == null) {
                 result.addValidationError(new StandardValidationError(concat(fieldPrefix, field), rawValue, ValidationErrorType.INVALID_FIELD,
-                    "Value is of type " + rawValue.getClass().getName() + " but was expected to be of type " + dataType));
+                    "Value is of type " + classNameOrNull(rawValue) + " but was expected to be of type " + dataType));
 
                 return null;
             }
@@ -157,7 +157,7 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
         if (canonicalDataType.getFieldType() == RecordFieldType.RECORD) {
             if (!(rawValue instanceof Record)) { // sanity check
                 result.addValidationError(new StandardValidationError(concat(fieldPrefix, field), rawValue, ValidationErrorType.INVALID_FIELD,
-                    "Value is of type " + rawValue.getClass().getName() + " but was expected to be of type " + expectedDataType));
+                    "Value is of type " + classNameOrNull(rawValue) + " but was expected to be of type " + expectedDataType));
 
                 return;
             }
@@ -189,6 +189,9 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
 
                 final Object[] array = (Object[]) value;
                 for (final Object arrayVal : array) {
+                    if (arrayVal == null && arrayDataType.isElementsNullable()) {
+                        continue;
+                    }
                     if (!isTypeCorrect(arrayVal, elementType)) {
                         return false;
                     }
@@ -202,6 +205,9 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
                     final Map<?, ?> map = (Map<?, ?>) value;
 
                     for (final Object mapValue : map.values()) {
+                        if (mapValue == null && mapDataType.isValuesNullable()) {
+                            continue;
+                        }
                         if (!isTypeCorrect(mapValue, valueDataType)) {
                             return false;
                         }
@@ -287,4 +293,8 @@ public class StandardSchemaValidator implements RecordSchemaValidator {
     private String concat(final String fieldPrefix, final RecordField field) {
         return fieldPrefix + "/" + field.getFieldName();
     }
+
+    private String classNameOrNull(Object value) {
+        return value == null ? "null" : value.getClass().getName();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
index 895e3de..dfcb41a 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java
@@ -640,4 +640,38 @@ public class TestValidateRecord {
                 + "The following 1 fields had values whose type did not match the schema: [/id]");
     }
 
+    @Test
+    public void testValidationForNullElementArrayAndMap() throws Exception {
+        AvroReader avroReader = new AvroReader();
+        runner.addControllerService("reader", avroReader);
+        runner.enableControllerService(avroReader);
+
+
+        final MockRecordWriter validWriter = new MockRecordWriter("valid", false);
+        runner.addControllerService("writer", validWriter);
+        runner.enableControllerService(validWriter);
+
+        final MockRecordWriter invalidWriter = new MockRecordWriter("invalid", true);
+        runner.addControllerService("invalid-writer", invalidWriter);
+        runner.enableControllerService(invalidWriter);
+
+        runner.setProperty(ValidateRecord.RECORD_READER, "reader");
+        runner.setProperty(ValidateRecord.RECORD_WRITER, "writer");
+        runner.setProperty(ValidateRecord.INVALID_RECORD_WRITER, "invalid-writer");
+        runner.setProperty(ValidateRecord.ALLOW_EXTRA_FIELDS, "false");
+        runner.setProperty(ValidateRecord.MAX_VALIDATION_DETAILS_LENGTH, "150");
+        runner.setProperty(ValidateRecord.VALIDATION_DETAILS_ATTRIBUTE_NAME, "valDetails");
+
+        runner.enqueue(Paths.get("src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro"));
+        runner.run();
+
+        runner.assertTransferCount(ValidateRecord.REL_INVALID, 0);
+        runner.assertTransferCount(ValidateRecord.REL_FAILURE, 0);
+        runner.assertTransferCount(ValidateRecord.REL_VALID, 1);
+
+        final MockFlowFile validFlowFile = runner.getFlowFilesForRelationship(ValidateRecord.REL_VALID).get(0);
+        validFlowFile.assertAttributeEquals("record.count", "1");
+        validFlowFile.assertContentEquals("valid\n[text, null],{key=null}\n");
+    }
+
 }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro
new file mode 100644
index 0000000..b9eb1d5
Binary files /dev/null and b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro differ