diff --git a/nifi-commons/nifi-json-schema-shared/pom.xml b/nifi-commons/nifi-json-schema-shared/pom.xml index c712e57192db..c1adb1ad19fc 100644 --- a/nifi-commons/nifi-json-schema-shared/pom.xml +++ b/nifi-commons/nifi-json-schema-shared/pom.xml @@ -31,5 +31,19 @@ nifi-json-schema-api 2.7.0-SNAPSHOT + + org.apache.nifi + nifi-record + ${project.version} + + + com.fasterxml.jackson.core + jackson-databind + + + org.apache.nifi + nifi-standard-record-utils + ${project.version} + - \ No newline at end of file + diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverter.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverter.java new file mode 100644 index 000000000000..56b2a593f779 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverter.java @@ -0,0 +1,715 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.record; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.node.ArrayNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.nifi.json.schema.validation.AdditionalPropertiesValidator; +import org.apache.nifi.json.schema.validation.ArrayLengthValidator; +import org.apache.nifi.json.schema.validation.ConstValidator; +import org.apache.nifi.json.schema.validation.EnumValidator; +import org.apache.nifi.json.schema.validation.MultipleOfValidator; +import org.apache.nifi.json.schema.validation.NumericRangeValidator; +import org.apache.nifi.json.schema.validation.PatternPropertiesValidator; +import org.apache.nifi.json.schema.validation.PatternValidator; +import org.apache.nifi.json.schema.validation.PropertyCountValidator; +import org.apache.nifi.json.schema.validation.PropertyNameValidator; +import org.apache.nifi.json.schema.validation.RequiredFieldPresenceValidator; +import org.apache.nifi.json.schema.validation.StringLengthValidator; +import org.apache.nifi.json.schema.validation.UniqueItemsValidator; +import org.apache.nifi.json.schema.validation.ValidatorUtils; +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.DataType; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; +import org.apache.nifi.serialization.record.type.ArrayDataType; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.RecordValidator; + +import java.math.BigDecimal; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Deque; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; +import java.util.stream.Collectors; +import java.util.function.Supplier; + +/** + * Converts JSON Schema definitions into NiFi {@link RecordSchema} instances while preserving validation rules. + */ +public class JsonSchemaToRecordSchemaConverter { + private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); + private static final String JSON_SCHEMA_FORMAT = "json-schema"; + private static final String TYPE_NULL = "null"; + private static final String TYPE_OBJECT = "object"; + private static final String TYPE_ARRAY = "array"; + private static final String TYPE_STRING = "string"; + private static final String TYPE_INTEGER = "integer"; + private static final String TYPE_NUMBER = "number"; + private static final String TYPE_BOOLEAN = "boolean"; + + public RecordSchema convert(final String schemaText) { + try { + final JsonNode rootNode = OBJECT_MAPPER.readTree(schemaText); + return convert(rootNode, schemaText); + } catch (final JsonProcessingException e) { + throw new IllegalArgumentException("Unable to parse JSON Schema", e); + } + } + + public RecordSchema convert(final JsonNode schemaNode, final String schemaText) { + if (!schemaNode.isObject()) { + throw new IllegalArgumentException("Root JSON Schema must be an object"); + } + + final ConversionContext context = new ConversionContext(schemaNode); + final ConversionResult conversionResult = convertObjectSchema(schemaNode, "", context); + final SimpleRecordSchema recordSchema = new SimpleRecordSchema(conversionResult.fields(), schemaText, JSON_SCHEMA_FORMAT, SchemaIdentifier.EMPTY); + recordSchema.setRecordValidators(conversionResult.recordValidators()); + + final String schemaName = textOrNull(schemaNode.get("title")); + if (schemaName != null) { + recordSchema.setSchemaName(schemaName); + } + + final String schemaNamespace = textOrNull(schemaNode.get("$id")); + if (schemaNamespace != null) { + recordSchema.setSchemaNamespace(schemaNamespace); + } + + return recordSchema; + } + + private ConversionResult convertObjectSchema(final JsonNode schemaNode, final String fieldPath, final ConversionContext context) { + final ResolvedSchema resolved = resolveSchemaNode(schemaNode, context, fieldPath); + final JsonNode effectiveSchema = resolved.schemaNode(); + final String pointer = resolved.pointer(); + + if (pointer != null) { + return context.resolveObjectReference(pointer, () -> convertObjectSchemaInternal(effectiveSchema, fieldPath, context)); + } + + return convertObjectSchemaInternal(effectiveSchema, fieldPath, context); + } + + private ConversionResult convertObjectSchemaInternal(final JsonNode schemaNode, final String fieldPath, final ConversionContext context) { + final List fields = new ArrayList<>(); + final List recordValidators = new ArrayList<>(); + + final Set requiredFields = parseRequiredFields(schemaNode); + final Set requiredNullableFields = new LinkedHashSet<>(); + + // TODO: Merge combinator keywords (allOf/anyOf/oneOf) into the synthesized schema. + final JsonNode propertiesNode = schemaNode.get("properties"); + if (propertiesNode instanceof ObjectNode objectNode) { + final Iterator> fieldsIterator = objectNode.fields(); + while (fieldsIterator.hasNext()) { + final Map.Entry entry = fieldsIterator.next(); + final String fieldName = entry.getKey(); + final JsonNode fieldSchema = entry.getValue(); + final boolean required = requiredFields.contains(fieldName); + final String childPath = ValidatorUtils.buildFieldPath(fieldPath, fieldName); + final FieldCreationResult fieldCreationResult = createRecordField(fieldName, fieldSchema, required, childPath, context); + fields.add(fieldCreationResult.recordField()); + if (fieldCreationResult.requiredButNullable()) { + requiredNullableFields.add(fieldName); + } + } + } + + final List patternPropertyDefinitions = new ArrayList<>(); + final JsonNode patternPropertiesNode = schemaNode.get("patternProperties"); + if (patternPropertiesNode instanceof ObjectNode patternPropertiesObject) { + final Iterator> patternsIterator = patternPropertiesObject.fields(); + while (patternsIterator.hasNext()) { + final Map.Entry entry = patternsIterator.next(); + final String patternExpression = entry.getKey(); + final Pattern compiledPattern; + try { + compiledPattern = Pattern.compile(patternExpression); + } catch (final PatternSyntaxException e) { + throw new IllegalArgumentException("Invalid regex pattern in patternProperties: " + patternExpression, e); + } + + final JsonNode patternSchema = entry.getValue(); + final String childPath = ValidatorUtils.buildFieldPath(fieldPath, patternExpression); + final FieldCreationResult patternResult = createRecordField(patternExpression, patternSchema, false, childPath, context); + patternPropertyDefinitions.add(new PatternPropertiesValidator.PatternPropertyDefinition(compiledPattern, patternResult.recordField())); + } + } + + if (schemaNode.has("additionalProperties") && schemaNode.get("additionalProperties").isBoolean() + && !schemaNode.get("additionalProperties").booleanValue()) { + final Set allowedFields = fields.stream().map(RecordField::getFieldName).collect(Collectors.toUnmodifiableSet()); + final List allowedPatterns = patternPropertyDefinitions.stream() + .map(PatternPropertiesValidator.PatternPropertyDefinition::pattern) + .collect(Collectors.toUnmodifiableList()); + recordValidators.add(new AdditionalPropertiesValidator(allowedFields, allowedPatterns)); + } + + final Integer minProperties = schemaNode.has("minProperties") ? Integer.valueOf(schemaNode.get("minProperties").intValue()) : null; + final Integer maxProperties = schemaNode.has("maxProperties") ? Integer.valueOf(schemaNode.get("maxProperties").intValue()) : null; + if (minProperties != null || maxProperties != null) { + recordValidators.add(new PropertyCountValidator(minProperties, maxProperties)); + } + + if (!patternPropertyDefinitions.isEmpty()) { + recordValidators.add(new PatternPropertiesValidator(patternPropertyDefinitions)); + } + + final JsonNode propertyNamesNode = schemaNode.get("propertyNames"); + if (propertyNamesNode != null) { + final PropertyNameValidator propertyNameValidator = createPropertyNameValidator(propertyNamesNode, fieldPath, context); + if (propertyNameValidator != null) { + recordValidators.add(propertyNameValidator); + } + } + + if (!requiredNullableFields.isEmpty()) { + recordValidators.add(new RequiredFieldPresenceValidator(requiredNullableFields)); + } + + return new ConversionResult(List.copyOf(fields), List.copyOf(recordValidators)); + } + + private FieldCreationResult createRecordField(final String fieldName, final JsonNode fieldSchema, final boolean required, final String fieldPath, final ConversionContext context) { + final List fieldValidators = new ArrayList<>(); + + final ResolvedSchema resolvedFieldSchema = resolveSchemaNode(fieldSchema, context, fieldPath); + final JsonNode effectiveFieldSchema = resolvedFieldSchema.schemaNode(); + + final List typeTokens = new ArrayList<>(extractTypes(effectiveFieldSchema)); + final boolean explicitNull = typeTokens.remove(TYPE_NULL); + final boolean nullable = !required || explicitNull; + + DataType dataType = determineDataType(effectiveFieldSchema, fieldPath, typeTokens, fieldValidators, context, resolvedFieldSchema.pointer()); + if (dataType == null) { + dataType = RecordFieldType.STRING.getDataType(); + } + + applyStringConstraints(effectiveFieldSchema, typeTokens, fieldValidators); + applyNumericConstraints(effectiveFieldSchema, typeTokens, fieldValidators); + applyArrayConstraints(effectiveFieldSchema, typeTokens, fieldValidators); + dataType = applyEnumConstraints(effectiveFieldSchema, typeTokens, fieldValidators, dataType); + + if (effectiveFieldSchema.has("const")) { + final Object constantValue = convertJsonNode(effectiveFieldSchema.get("const")); + fieldValidators.add(new ConstValidator(constantValue)); + } + + final Object defaultValue = extractDefaultValue(effectiveFieldSchema, dataType); + final RecordField recordField = new RecordField(fieldName, dataType, defaultValue, Collections.emptySet(), nullable, fieldValidators); + final boolean requiredButNullable = required && nullable && explicitNull; + return new FieldCreationResult(recordField, requiredButNullable); + } + + private DataType determineDataType(final JsonNode fieldSchema, final String fieldPath, final List typeTokens, + final List fieldValidators, final ConversionContext context, final String schemaPointer) { + if (typeTokens.isEmpty()) { + inferTypeWhenMissing(fieldSchema, typeTokens); + } + + if (typeTokens.size() > 1) { + final List childTypes = typeTokens.stream() + .map(typeToken -> createDataTypeForToken(typeToken, fieldSchema, fieldPath, fieldValidators, context, schemaPointer)) + .filter(Objects::nonNull) + .collect(Collectors.toList()); + return RecordFieldType.CHOICE.getChoiceDataType(childTypes); + } else if (!typeTokens.isEmpty()) { + return createDataTypeForToken(typeTokens.get(0), fieldSchema, fieldPath, fieldValidators, context, schemaPointer); + } + + return null; + } + + private void inferTypeWhenMissing(final JsonNode fieldSchema, final List typeTokens) { + if (fieldSchema.has("properties") || fieldSchema.has("required")) { + typeTokens.add(TYPE_OBJECT); + } else if (fieldSchema.has("items")) { + typeTokens.add(TYPE_ARRAY); + } else if (fieldSchema.has("enum")) { + typeTokens.add(TYPE_STRING); + } + } + + private DataType createDataTypeForToken(final String typeToken, final JsonNode fieldSchema, final String fieldPath, + final List fieldValidators, final ConversionContext context, final String schemaPointer) { + return switch (typeToken) { + case TYPE_OBJECT -> createRecordDataType(fieldSchema, fieldPath, context, schemaPointer); + case TYPE_ARRAY -> createArrayDataType(fieldSchema, fieldPath, fieldValidators, context); + case TYPE_STRING -> createStringDataType(fieldSchema); + case TYPE_INTEGER -> RecordFieldType.LONG.getDataType(); + case TYPE_NUMBER -> RecordFieldType.DOUBLE.getDataType(); + case TYPE_BOOLEAN -> RecordFieldType.BOOLEAN.getDataType(); + default -> RecordFieldType.STRING.getDataType(); + }; + } + + private DataType createRecordDataType(final JsonNode fieldSchema, final String fieldPath, final ConversionContext context, final String schemaPointer) { + final ConversionResult nestedResult; + if (schemaPointer != null) { + nestedResult = context.resolveObjectReference(schemaPointer, () -> convertObjectSchemaInternal(fieldSchema, fieldPath, context)); + } else { + nestedResult = convertObjectSchemaInternal(fieldSchema, fieldPath, context); + } + final SimpleRecordSchema childSchema = new SimpleRecordSchema(nestedResult.fields()); + childSchema.setRecordValidators(nestedResult.recordValidators()); + return RecordFieldType.RECORD.getRecordDataType(childSchema); + } + + private DataType createArrayDataType(final JsonNode fieldSchema, final String fieldPath, final List fieldValidators, final ConversionContext context) { + final JsonNode itemsNode = fieldSchema.get("items"); + final DataType elementType; + boolean elementsNullable = false; + if (itemsNode != null && itemsNode.isObject()) { + final String elementPath = fieldPath + "[]"; + final List elementValidators = new ArrayList<>(); + final ResolvedSchema resolvedItems = resolveSchemaNode(itemsNode, context, elementPath); + final JsonNode effectiveItemsNode = resolvedItems.schemaNode(); + final List elementTypes = new ArrayList<>(extractTypes(effectiveItemsNode)); + elementsNullable = elementTypes.remove(TYPE_NULL); + DataType determinedElementType = determineDataType(effectiveItemsNode, elementPath, elementTypes, elementValidators, context, resolvedItems.pointer()); + if (determinedElementType == null) { + determinedElementType = RecordFieldType.STRING.getDataType(); + } + applyStringConstraints(effectiveItemsNode, elementTypes, elementValidators); + applyNumericConstraints(effectiveItemsNode, elementTypes, elementValidators); + determinedElementType = applyEnumConstraints(effectiveItemsNode, elementTypes, elementValidators, determinedElementType); + + // Element-level validators are currently not surfaced separately; future enhancement could expose them via complex data types. + elementType = determinedElementType; + } else { + // TODO: Support tuple typing where items is an array of schemas with positional semantics. + elementType = RecordFieldType.STRING.getDataType(); + } + + final ArrayDataType arrayDataType = (ArrayDataType) RecordFieldType.ARRAY.getArrayDataType(elementType, elementsNullable); + + if (fieldSchema.has("minItems") || fieldSchema.has("maxItems")) { + final Integer minItems = fieldSchema.has("minItems") ? fieldSchema.get("minItems").intValue() : null; + final Integer maxItems = fieldSchema.has("maxItems") ? fieldSchema.get("maxItems").intValue() : null; + fieldValidators.add(new ArrayLengthValidator(minItems, maxItems)); + } + + if (fieldSchema.path("uniqueItems").asBoolean(false)) { + fieldValidators.add(new UniqueItemsValidator()); + } + + return arrayDataType; + } + + private PropertyNameValidator createPropertyNameValidator(final JsonNode propertyNamesNode, final String fieldPath, final ConversionContext context) { + final ResolvedSchema resolved = resolveSchemaNode(propertyNamesNode, context, fieldPath + "/propertyNames"); + final JsonNode effectiveSchema = resolved.schemaNode(); + if (effectiveSchema == null || effectiveSchema.isNull()) { + return null; + } + + final Set supportedKeywords = Set.of( + "type", "enum", "pattern", "minLength", "maxLength", "const", + "description", "title", "default", "examples" + ); + + effectiveSchema.fieldNames().forEachRemaining(keyword -> { + if (!supportedKeywords.contains(keyword)) { + throw new IllegalArgumentException("Unsupported keyword '" + keyword + "' in propertyNames schema at " + fieldPath); + } + }); + + final List typeTokens = new ArrayList<>(extractTypes(effectiveSchema)); + if (!typeTokens.isEmpty()) { + if (typeTokens.size() > 1 || !TYPE_STRING.equals(typeTokens.get(0))) { + throw new IllegalArgumentException("propertyNames schema at " + fieldPath + " must target string type"); + } + } + + Pattern pattern = null; + if (effectiveSchema.has("pattern")) { + pattern = Pattern.compile(effectiveSchema.get("pattern").textValue()); + } + + Integer minLength = null; + if (effectiveSchema.has("minLength")) { + minLength = effectiveSchema.get("minLength").intValue(); + if (minLength < 0) { + throw new IllegalArgumentException("propertyNames minLength must be non-negative at " + fieldPath); + } + } + + Integer maxLength = null; + if (effectiveSchema.has("maxLength")) { + maxLength = effectiveSchema.get("maxLength").intValue(); + if (maxLength < 0) { + throw new IllegalArgumentException("propertyNames maxLength must be non-negative at " + fieldPath); + } + } + + if (minLength != null && maxLength != null && minLength > maxLength) { + throw new IllegalArgumentException("propertyNames minLength cannot exceed maxLength at " + fieldPath); + } + + String constValue = null; + if (effectiveSchema.has("const")) { + final JsonNode constNode = effectiveSchema.get("const"); + if (!constNode.isTextual()) { + throw new IllegalArgumentException("propertyNames const must be a string at " + fieldPath); + } + constValue = constNode.textValue(); + } + + Set enumValues = Collections.emptySet(); + if (effectiveSchema.has("enum")) { + final JsonNode enumNode = effectiveSchema.get("enum"); + if (!enumNode.isArray()) { + throw new IllegalArgumentException("propertyNames enum must be an array at " + fieldPath); + } + + final Set values = new LinkedHashSet<>(); + enumNode.forEach(valueNode -> { + if (!valueNode.isTextual()) { + throw new IllegalArgumentException("propertyNames enum values must be strings at " + fieldPath); + } + values.add(valueNode.textValue()); + }); + enumValues = values; + } + + if (pattern == null && minLength == null && maxLength == null && enumValues.isEmpty() && constValue == null) { + return null; + } + + final List descriptionParts = new ArrayList<>(); + if (pattern != null) { + descriptionParts.add("pattern=" + pattern.pattern()); + } + if (minLength != null) { + descriptionParts.add("minLength=" + minLength); + } + if (maxLength != null) { + descriptionParts.add("maxLength=" + maxLength); + } + if (!enumValues.isEmpty()) { + descriptionParts.add("enum=" + enumValues); + } + if (constValue != null) { + descriptionParts.add("const='" + constValue + "'"); + } + + final String description = descriptionParts.isEmpty() + ? "propertyNames validator" + : "propertyNames constraints: " + String.join(", ", descriptionParts); + + return new PropertyNameValidator(minLength, maxLength, pattern, enumValues, constValue, description); + } + private DataType createStringDataType(final JsonNode fieldSchema) { + final String format = textOrNull(fieldSchema.get("format")); + if (format == null) { + return RecordFieldType.STRING.getDataType(); + } + + return switch (format) { + case "date" -> RecordFieldType.DATE.getDataType(null); + case "time" -> RecordFieldType.TIME.getDataType(null); + case "date-time" -> RecordFieldType.TIMESTAMP.getDataType(null); + case "uuid" -> RecordFieldType.UUID.getDataType(); + default -> RecordFieldType.STRING.getDataType(); + }; + } + + private void applyStringConstraints(final JsonNode fieldSchema, final List typeTokens, final List fieldValidators) { + if (!typeTokens.contains(TYPE_STRING)) { + return; + } + + final Integer minLength = fieldSchema.has("minLength") ? fieldSchema.get("minLength").intValue() : null; + final Integer maxLength = fieldSchema.has("maxLength") ? fieldSchema.get("maxLength").intValue() : null; + if (minLength != null || maxLength != null) { + fieldValidators.add(new StringLengthValidator(minLength, maxLength)); + } + + if (fieldSchema.has("pattern")) { + fieldValidators.add(new PatternValidator(fieldSchema.get("pattern").textValue())); + } + } + + private void applyNumericConstraints(final JsonNode fieldSchema, final List typeTokens, final List fieldValidators) { + if (!typeTokens.contains(TYPE_INTEGER) && !typeTokens.contains(TYPE_NUMBER)) { + return; + } + + BigDecimal minimum = fieldSchema.has("minimum") ? fieldSchema.get("minimum").decimalValue() : null; + BigDecimal maximum = fieldSchema.has("maximum") ? fieldSchema.get("maximum").decimalValue() : null; + boolean exclusiveMinimum = false; + boolean exclusiveMaximum = false; + + final JsonNode exclusiveMinimumNode = fieldSchema.get("exclusiveMinimum"); + if (exclusiveMinimumNode != null) { + if (exclusiveMinimumNode.isNumber()) { + minimum = exclusiveMinimumNode.decimalValue(); + exclusiveMinimum = true; + } else if (exclusiveMinimumNode.isBoolean()) { + exclusiveMinimum = exclusiveMinimumNode.booleanValue(); + } + } + + final JsonNode exclusiveMaximumNode = fieldSchema.get("exclusiveMaximum"); + if (exclusiveMaximumNode != null) { + if (exclusiveMaximumNode.isNumber()) { + maximum = exclusiveMaximumNode.decimalValue(); + exclusiveMaximum = true; + } else if (exclusiveMaximumNode.isBoolean()) { + exclusiveMaximum = exclusiveMaximumNode.booleanValue(); + } + } + + if (minimum != null || maximum != null || exclusiveMinimum || exclusiveMaximum) { + fieldValidators.add(new NumericRangeValidator(minimum, exclusiveMinimum, maximum, exclusiveMaximum)); + } + + if (fieldSchema.has("multipleOf")) { + fieldValidators.add(new MultipleOfValidator(fieldSchema.get("multipleOf").decimalValue())); + } + + // TODO: Enforce numeric format keywords (e.g., int32, int64, float, double) when provided. + } + + private void applyArrayConstraints(final JsonNode fieldSchema, final List typeTokens, final List fieldValidators) { + if (!typeTokens.contains(TYPE_ARRAY)) { + return; + } + // Array-specific validators are applied during array data type creation. + } + + private DataType applyEnumConstraints(final JsonNode fieldSchema, final List typeTokens, final List fieldValidators, final DataType currentDataType) { + if (!fieldSchema.has("enum")) { + return currentDataType; + } + + final ArrayNode enumNode = (ArrayNode) fieldSchema.get("enum"); + if (enumNode.isEmpty()) { + return currentDataType; + } + + final List enumValues = new ArrayList<>(); + enumNode.forEach(enumEntry -> enumValues.add(convertJsonNode(enumEntry))); + final boolean allStrings = enumValues.stream().allMatch(String.class::isInstance); + if (allStrings && (typeTokens.isEmpty() || (typeTokens.size() == 1 && TYPE_STRING.equals(typeTokens.get(0))))) { + final List stringValues = enumValues.stream().map(Object::toString).collect(Collectors.toList()); + fieldValidators.add(new EnumValidator(stringValues)); + return RecordFieldType.ENUM.getEnumDataType(stringValues); + } + + final Collection canonicalValues = EnumValidator.canonicalizeAll(enumValues); + fieldValidators.add(new EnumValidator(canonicalValues)); + return currentDataType; + } + + private Object extractDefaultValue(final JsonNode fieldSchema, final DataType dataType) { + final JsonNode defaultNode = fieldSchema.get("default"); + if (defaultNode == null || defaultNode.isNull()) { + return null; + } + + final Object defaultValue = convertJsonNode(defaultNode); + if (defaultValue == null) { + return null; + } + + final RecordFieldType fieldType = dataType.getFieldType(); + if (fieldType == RecordFieldType.BOOLEAN && defaultValue instanceof Boolean) { + return defaultValue; + } + if (fieldType == RecordFieldType.STRING && defaultValue instanceof String) { + return defaultValue; + } + if ((fieldType == RecordFieldType.INT || fieldType == RecordFieldType.LONG || fieldType == RecordFieldType.BIGINT + || fieldType == RecordFieldType.FLOAT || fieldType == RecordFieldType.DOUBLE || fieldType == RecordFieldType.DECIMAL) + && defaultValue instanceof Number) { + return defaultValue; + } + + return null; + } + + private Set parseRequiredFields(final JsonNode schemaNode) { + if (!schemaNode.has("required")) { + return Set.of(); + } + + final ArrayNode requiredArray = (ArrayNode) schemaNode.get("required"); + final Set requiredFields = new LinkedHashSet<>(); + requiredArray.forEach(node -> requiredFields.add(node.asText())); + return requiredFields; + } + + private List extractTypes(final JsonNode schemaNode) { + if (schemaNode == null || !schemaNode.has("type")) { + return new ArrayList<>(); + } + + final JsonNode typeNode = schemaNode.get("type"); + if (typeNode.isTextual()) { + return new ArrayList<>(List.of(typeNode.textValue())); + } + + if (typeNode.isArray()) { + final ArrayNode arrayNode = (ArrayNode) typeNode; + final List types = new ArrayList<>(); + arrayNode.forEach(node -> types.add(node.textValue())); + return types; + } + + return new ArrayList<>(); + } + + private ResolvedSchema resolveSchemaNode(final JsonNode schemaNode, final ConversionContext context, final String fieldPath) { + if (schemaNode == null) { + return new ResolvedSchema(null, null); + } + + JsonNode currentNode = schemaNode; + String pointerForCache = null; + final Set visitedPointers = new HashSet<>(); + + while (currentNode.has("$ref")) { + final JsonNode refNode = currentNode.get("$ref"); + if (!refNode.isTextual()) { + throw new IllegalArgumentException("Schema $ref must be a string at " + fieldPath); + } + + final String ref = refNode.textValue(); + if (ref == null || ref.isBlank()) { + throw new IllegalArgumentException("Schema $ref must be a non-empty string at " + fieldPath); + } + + if (!ref.startsWith("#")) { + throw new IllegalArgumentException("External JSON Schema references are not supported: " + ref); + } + + if (currentNode.size() > 1) { + throw new IllegalArgumentException("Schema at " + fieldPath + " combines $ref with other keywords which is not supported"); + } + + final String pointer = ref.equals("#") ? "" : ref.substring(1); + if (!pointer.isEmpty() && !pointer.startsWith("/")) { + throw new IllegalArgumentException("Unsupported JSON Pointer syntax in reference '" + ref + "' at " + fieldPath); + } + + final String cacheKey = pointer.isEmpty() ? "#" : pointer; + if (!visitedPointers.add(cacheKey)) { + throw new IllegalArgumentException("Detected circular schema reference involving pointer '" + cacheKey + "' at " + fieldPath); + } + + final JsonNode target = pointer.isEmpty() ? context.rootNode : context.rootNode.at(pointer); + if (target.isMissingNode()) { + throw new IllegalArgumentException("Unable to resolve JSON Schema reference '" + ref + "' at " + fieldPath); + } + + pointerForCache = cacheKey; + currentNode = target; + } + + return new ResolvedSchema(currentNode, pointerForCache); + } + + private Object convertJsonNode(final JsonNode node) { + if (node == null || node.isNull()) { + return null; + } + if (node.isTextual()) { + return node.textValue(); + } + if (node.isBoolean()) { + return node.booleanValue(); + } + if (node.isInt() || node.isLong()) { + return node.longValue(); + } + if (node.isFloatingPointNumber()) { + return node.doubleValue(); + } + if (node.isArray()) { + final ArrayNode arrayNode = (ArrayNode) node; + final List values = new ArrayList<>(arrayNode.size()); + arrayNode.forEach(element -> values.add(convertJsonNode(element))); + return values.toArray(); + } + if (node.isObject()) { + return OBJECT_MAPPER.convertValue(node, Map.class); + } + return null; + } + + private String textOrNull(final JsonNode node) { + return node != null && node.isTextual() ? node.textValue() : null; + } + + private record ConversionResult(List fields, List recordValidators) { + } + + private record FieldCreationResult(RecordField recordField, boolean requiredButNullable) { + } + + private static final class ConversionContext { + private final JsonNode rootNode; + private final Map objectSchemaCache = new HashMap<>(); + private final Deque resolutionStack = new ArrayDeque<>(); + + private ConversionContext(final JsonNode rootNode) { + this.rootNode = Objects.requireNonNull(rootNode); + } + + private ConversionResult resolveObjectReference(final String pointer, final Supplier converter) { + final ConversionResult cached = objectSchemaCache.get(pointer); + if (cached != null) { + return cached; + } + + if (resolutionStack.contains(pointer)) { + throw new IllegalArgumentException("Detected circular schema reference for pointer '" + pointer + "'"); + } + + resolutionStack.push(pointer); + try { + final ConversionResult result = converter.get(); + objectSchemaCache.put(pointer, result); + return result; + } finally { + resolutionStack.pop(); + } + } + } + + private record ResolvedSchema(JsonNode schemaNode, String pointer) { + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/AdditionalPropertiesValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/AdditionalPropertiesValidator.java new file mode 100644 index 000000000000..2a9b111aec39 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/AdditionalPropertiesValidator.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.validation.RecordValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * Enforces the JSON Schema {@code additionalProperties: false} constraint. + */ +public class AdditionalPropertiesValidator implements RecordValidator { + private final Set allowedFields; + private final List allowedPatterns; + + public AdditionalPropertiesValidator(final Set allowedFields) { + this(allowedFields, List.of()); + } + + public AdditionalPropertiesValidator(final Set allowedFields, final Collection allowedPatterns) { + this.allowedFields = Set.copyOf(allowedFields); + this.allowedPatterns = List.copyOf(allowedPatterns); + } + + @Override + public Collection validate(final Record record, final RecordSchema schema, final String fieldPath) { + final Collection errors = new ArrayList<>(); + for (final String rawFieldName : record.getRawFieldNames()) { + if (!allowedFields.contains(rawFieldName) && allowedPatterns.stream().noneMatch(pattern -> pattern.matcher(rawFieldName).find())) { + final String fullFieldPath = ValidatorUtils.buildFieldPath(fieldPath, rawFieldName); + final ValidationError error = ValidatorUtils.createError(fullFieldPath, record.getValue(rawFieldName), ValidationErrorType.EXTRA_FIELD, + "Field is not defined in schema"); + errors.add(error); + } + } + return errors; + } + + @Override + public String getDescription() { + return "Additional properties disallowed"; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ArrayLengthValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ArrayLengthValidator.java new file mode 100644 index 000000000000..561ba5cc22da --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ArrayLengthValidator.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; + +/** + * Validates {@code minItems} and {@code maxItems} constraints for array fields. + */ +public class ArrayLengthValidator implements FieldValidator { + private final Integer minItems; + private final Integer maxItems; + + public ArrayLengthValidator(final Integer minItems, final Integer maxItems) { + this.minItems = minItems; + this.maxItems = maxItems; + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final int size = determineSize(value); + if (size < 0) { + return ValidatorUtils.errorCollection(null); + } + + if (minItems != null && size < minItems) { + final String explanation = String.format("Array must contain at least %d items", minItems); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + if (maxItems != null && size > maxItems) { + final String explanation = String.format("Array must contain no more than %d items", maxItems); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + final StringBuilder description = new StringBuilder("Array length validator"); + if (minItems != null) { + description.append(", minItems=").append(minItems); + } + if (maxItems != null) { + description.append(", maxItems=").append(maxItems); + } + return description.toString(); + } + + private int determineSize(final Object value) { + if (value == null) { + return -1; + } + + if (value instanceof Object[]) { + return ((Object[]) value).length; + } + + if (value instanceof java.util.Collection) { + return ((java.util.Collection) value).size(); + } + + return -1; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ConstValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ConstValidator.java new file mode 100644 index 000000000000..cf5621066ff5 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ConstValidator.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; + +/** + * Validates that the field value matches the JSON Schema {@code const} keyword. + */ +public class ConstValidator implements FieldValidator { + private final String canonicalValue; + + public ConstValidator(final Object constantValue) { + this.canonicalValue = EnumValidator.canonicalize(constantValue); + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final String candidate = EnumValidator.canonicalize(value); + if (!canonicalValue.equals(candidate)) { + final String explanation = String.format("Value must equal %s", canonicalValue); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + return "Const validator: value=" + canonicalValue; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/EnumValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/EnumValidator.java new file mode 100644 index 000000000000..bdcf0ab0be63 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/EnumValidator.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * Validates a value against a finite set of allowed values as defined by the JSON Schema {@code enum} keyword. + */ +public class EnumValidator implements FieldValidator { + private final Set allowedValues; + private final String description; + + public EnumValidator(final Collection allowedValues) { + this.allowedValues = Set.copyOf(allowedValues); + this.description = "Enum validator" + allowedValues; + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final String canonicalValue = canonicalize(value); + if (!allowedValues.contains(canonicalValue)) { + final String explanation = String.format("Value must be one of %s", allowedValues); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + return description; + } + + public static Collection canonicalizeAll(final Collection values) { + return values.stream().map(EnumValidator::canonicalizeStatic).collect(Collectors.toSet()); + } + + public static String canonicalize(final Object value) { + return canonicalizeStatic(value); + } + + private static String canonicalizeStatic(final Object value) { + if (value == null) { + return "null"; + } + + final Class valueClass = value.getClass(); + if (valueClass.isArray()) { + if (value instanceof Object[]) { + return Arrays.deepToString((Object[]) value); + } + if (value instanceof byte[]) { + return Arrays.toString((byte[]) value); + } + if (value instanceof short[]) { + return Arrays.toString((short[]) value); + } + if (value instanceof int[]) { + return Arrays.toString((int[]) value); + } + if (value instanceof long[]) { + return Arrays.toString((long[]) value); + } + if (value instanceof float[]) { + return Arrays.toString((float[]) value); + } + if (value instanceof double[]) { + return Arrays.toString((double[]) value); + } + if (value instanceof char[]) { + return Arrays.toString((char[]) value); + } + if (value instanceof boolean[]) { + return Arrays.toString((boolean[]) value); + } + } + + return String.valueOf(value); + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/MultipleOfValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/MultipleOfValidator.java new file mode 100644 index 000000000000..bd9eca06736e --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/MultipleOfValidator.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Collection; + +/** + * Validates the JSON Schema {@code multipleOf} constraint for numeric fields. + */ +public class MultipleOfValidator implements FieldValidator { + private final BigDecimal divisor; + + public MultipleOfValidator(final BigDecimal divisor) { + if (divisor == null || BigDecimal.ZERO.compareTo(divisor) == 0) { + throw new IllegalArgumentException("multipleOf divisor must be a non-zero number"); + } + this.divisor = divisor.stripTrailingZeros(); + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final BigDecimal decimalValue = toBigDecimal(value); + if (decimalValue == null) { + return ValidatorUtils.errorCollection(null); + } + + final BigDecimal remainder = decimalValue.remainder(divisor); + if (remainder.compareTo(BigDecimal.ZERO) != 0) { + final String explanation = String.format("Value must be a multiple of %s", divisor); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + return "Multiple-of validator: divisor=" + divisor; + } + + private BigDecimal toBigDecimal(final Object value) { + if (value instanceof BigDecimal) { + return (BigDecimal) value; + } + if (value instanceof BigInteger) { + return new BigDecimal((BigInteger) value); + } + if (value instanceof Number) { + return BigDecimal.valueOf(((Number) value).doubleValue()); + } + return null; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/NumericRangeValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/NumericRangeValidator.java new file mode 100644 index 000000000000..9b99352d5059 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/NumericRangeValidator.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.math.BigDecimal; +import java.math.BigInteger; +import java.util.Collection; + +/** + * Validates numeric minimum and maximum constraints according to JSON Schema semantics. + */ +public class NumericRangeValidator implements FieldValidator { + private final BigDecimal minimum; + private final boolean exclusiveMinimum; + private final BigDecimal maximum; + private final boolean exclusiveMaximum; + + public NumericRangeValidator(final BigDecimal minimum, final boolean exclusiveMinimum, final BigDecimal maximum, final boolean exclusiveMaximum) { + this.minimum = minimum; + this.exclusiveMinimum = exclusiveMinimum; + this.maximum = maximum; + this.exclusiveMaximum = exclusiveMaximum; + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final BigDecimal decimalValue = toBigDecimal(value); + if (decimalValue == null) { + return ValidatorUtils.errorCollection(null); + } + + if (minimum != null) { + final int comparison = decimalValue.compareTo(minimum); + if (comparison < 0 || (exclusiveMinimum && comparison == 0)) { + final String explanation = exclusiveMinimum + ? String.format("Value must be greater than %s", minimum) + : String.format("Value must be greater than or equal to %s", minimum); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + } + + if (maximum != null) { + final int comparison = decimalValue.compareTo(maximum); + if (comparison > 0 || (exclusiveMaximum && comparison == 0)) { + final String explanation = exclusiveMaximum + ? String.format("Value must be less than %s", maximum) + : String.format("Value must be less than or equal to %s", maximum); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + final StringBuilder description = new StringBuilder("Numeric range validator"); + if (minimum != null) { + description.append(exclusiveMinimum ? ", exclusive min=" : ", min=").append(minimum); + } + if (maximum != null) { + description.append(exclusiveMaximum ? ", exclusive max=" : ", max=").append(maximum); + } + return description.toString(); + } + + private BigDecimal toBigDecimal(final Object value) { + if (value instanceof BigDecimal) { + return (BigDecimal) value; + } + if (value instanceof BigInteger) { + return new BigDecimal((BigInteger) value); + } + if (value instanceof Byte || value instanceof Short || value instanceof Integer || value instanceof Long) { + return BigDecimal.valueOf(((Number) value).longValue()); + } + if (value instanceof Float || value instanceof Double) { + return BigDecimal.valueOf(((Number) value).doubleValue()); + } + + return null; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidator.java new file mode 100644 index 000000000000..b8c9820367f6 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidator.java @@ -0,0 +1,141 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.MapRecord; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.type.RecordDataType; +import org.apache.nifi.serialization.record.validation.DefaultValidationError; +import org.apache.nifi.serialization.record.validation.RecordValidator; +import org.apache.nifi.serialization.record.validation.SchemaValidationResult; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.schema.validation.SchemaValidationContext; +import org.apache.nifi.schema.validation.StandardSchemaValidator; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * Applies JSON Schema {@code patternProperties} validation rules to records. + */ +public class PatternPropertiesValidator implements RecordValidator { + private final List patternDefinitions; + + public PatternPropertiesValidator(final Collection patternDefinitions) { + Objects.requireNonNull(patternDefinitions, "Pattern definitions required"); + this.patternDefinitions = List.copyOf(patternDefinitions); + } + + @Override + public Collection validate(final Record record, final RecordSchema schema, final String fieldPath) { + final List validationErrors = new ArrayList<>(); + final Set explicitFieldNames = Set.copyOf(schema.getFieldNames()); + + for (final PatternPropertyDefinition definition : patternDefinitions) { + final Pattern pattern = definition.pattern(); + + for (final String rawFieldName : record.getRawFieldNames()) { + if (explicitFieldNames.contains(rawFieldName)) { + continue; + } + + if (pattern.matcher(rawFieldName).find()) { + final Object value = record.getValue(rawFieldName); + final Collection errors = definition.validate(fieldPath, rawFieldName, value); + if (!errors.isEmpty()) { + validationErrors.addAll(errors); + } + } + } + } + + return validationErrors; + } + + @Override + public String getDescription() { + return "Pattern properties validator"; + } + + public record PatternPropertyDefinition(Pattern pattern, RecordField templateField) { + public PatternPropertyDefinition { + Objects.requireNonNull(pattern, "Pattern is required"); + Objects.requireNonNull(templateField, "Template field is required"); + } + + public Collection validate(final String basePath, final String actualFieldName, final Object value) { + final List collectedErrors = new ArrayList<>(); + + final RecordField actualField = new RecordField(actualFieldName, + templateField.getDataType(), + templateField.getDefaultValue(), + templateField.getAliases(), + templateField.isNullable(), + templateField.getFieldValidators()); + + final SimpleRecordSchema singleFieldSchema = new SimpleRecordSchema(List.of(actualField)); + final SchemaValidationContext context = new SchemaValidationContext(singleFieldSchema, true, false); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + Object normalizedValue = value; + if (actualField.getDataType().getFieldType() == RecordFieldType.RECORD && value instanceof Map mapValue) { + final RecordDataType recordDataType = (RecordDataType) actualField.getDataType(); + final RecordSchema childSchema = recordDataType.getChildSchema(); + final Map normalizedMap = new java.util.HashMap<>(); + for (final Map.Entry entry : mapValue.entrySet()) { + normalizedMap.put(String.valueOf(entry.getKey()), entry.getValue()); + } + normalizedValue = new MapRecord(childSchema, normalizedMap); + } + + final MapRecord singleRecord = new MapRecord(singleFieldSchema, Map.of(actualFieldName, normalizedValue)); + final SchemaValidationResult result = validator.validate(singleRecord); + + if (result.isValid()) { + return collectedErrors; + } + + for (final ValidationError error : result.getValidationErrors()) { + final Optional originalFieldPath = error.getFieldName(); + final String normalizedPath = originalFieldPath + .map(name -> name.startsWith("/") ? name.substring(1) : name) + .orElse(actualFieldName); + final String combinedPath = ValidatorUtils.buildFieldPath(basePath, normalizedPath); + + final DefaultValidationError rewritten = DefaultValidationError.builder() + .fieldName(combinedPath) + .inputValue(error.getInputValue().orElse(null)) + .type(error.getType()) + .explanation(error.getExplanation()) + .build(); + collectedErrors.add(rewritten); + } + + return collectedErrors; + } + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternValidator.java new file mode 100644 index 000000000000..4b5f0affe943 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PatternValidator.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; +import java.util.regex.Pattern; + +/** + * Validates values against the JSON Schema {@code pattern} expression. + */ +public class PatternValidator implements FieldValidator { + private final Pattern pattern; + + public PatternValidator(final String regex) { + this.pattern = Pattern.compile(regex); + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + if (!(value instanceof CharSequence)) { + return ValidatorUtils.errorCollection(null); + } + + final CharSequence text = (CharSequence) value; + if (!pattern.matcher(text).matches()) { + final String explanation = String.format("Value does not match pattern '%s'", pattern.pattern()); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + return "Pattern validator: pattern='" + pattern.pattern() + "'"; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyCountValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyCountValidator.java new file mode 100644 index 000000000000..a51a4f791afa --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyCountValidator.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.validation.RecordValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Validates {@code minProperties} and {@code maxProperties} constraints for JSON objects. + */ +public class PropertyCountValidator implements RecordValidator { + private final Integer minProperties; + private final Integer maxProperties; + + public PropertyCountValidator(final Integer minProperties, final Integer maxProperties) { + this.minProperties = minProperties; + this.maxProperties = maxProperties; + } + + @Override + public Collection validate(final Record record, final RecordSchema schema, final String fieldPath) { + final int propertyCount = record.getRawFieldNames().size(); + final Collection errors = new ArrayList<>(); + + if (minProperties != null && propertyCount < minProperties) { + final String explanation = String.format("Record must contain at least %d properties", minProperties); + errors.add(ValidatorUtils.createError(fieldPath, propertyCount, ValidationErrorType.OTHER, explanation)); + } + + if (maxProperties != null && propertyCount > maxProperties) { + final String explanation = String.format("Record must contain no more than %d properties", maxProperties); + errors.add(ValidatorUtils.createError(fieldPath, propertyCount, ValidationErrorType.OTHER, explanation)); + } + + return errors; + } + + @Override + public String getDescription() { + final StringBuilder description = new StringBuilder("Property count validator"); + if (minProperties != null) { + description.append(", minProperties=").append(minProperties); + } + if (maxProperties != null) { + description.append(", maxProperties=").append(maxProperties); + } + return description.toString(); + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyNameValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyNameValidator.java new file mode 100644 index 000000000000..6f1b229403dc --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/PropertyNameValidator.java @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.validation.RecordValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; +import java.util.regex.Pattern; + +/** + * Validates each property name of a record against JSON Schema {@code propertyNames} constraints. + */ +public class PropertyNameValidator implements RecordValidator { + private final Integer minLength; + private final Integer maxLength; + private final Pattern pattern; + private final Set allowedNames; + private final String constValue; + private final String description; + + public PropertyNameValidator(final Integer minLength, final Integer maxLength, final Pattern pattern, + final Set allowedNames, final String constValue, final String description) { + this.minLength = minLength; + this.maxLength = maxLength; + this.pattern = pattern; + this.allowedNames = allowedNames == null ? Collections.emptySet() : Set.copyOf(allowedNames); + this.constValue = constValue; + this.description = description; + } + + @Override + public Collection validate(final Record record, final RecordSchema schema, final String fieldPath) { + if (record == null) { + return Collections.emptyList(); + } + + final Collection errors = new ArrayList<>(); + for (final String propertyName : record.getRawFieldNames()) { + final ValidationError error = validatePropertyName(propertyName, fieldPath); + if (error != null) { + errors.add(error); + } + } + + return errors; + } + + @Override + public String getDescription() { + return description; + } + + private ValidationError validatePropertyName(final String propertyName, final String fieldPath) { + Objects.requireNonNull(propertyName, "Property name cannot be null"); + + if (constValue != null && !constValue.equals(propertyName)) { + return ValidatorUtils.createError(ValidatorUtils.buildFieldPath(fieldPath, propertyName), propertyName, + ValidationErrorType.INVALID_FIELD, "Property name must equal '" + constValue + "'"); + } + + if (!allowedNames.isEmpty() && !allowedNames.contains(propertyName)) { + return ValidatorUtils.createError(ValidatorUtils.buildFieldPath(fieldPath, propertyName), propertyName, + ValidationErrorType.INVALID_FIELD, "Property name is not one of the allowed values"); + } + + final int length = propertyName.length(); + if (minLength != null && length < minLength) { + return ValidatorUtils.createError(ValidatorUtils.buildFieldPath(fieldPath, propertyName), propertyName, + ValidationErrorType.INVALID_FIELD, "Property name length " + length + " is less than minimum " + minLength); + } + + if (maxLength != null && length > maxLength) { + return ValidatorUtils.createError(ValidatorUtils.buildFieldPath(fieldPath, propertyName), propertyName, + ValidationErrorType.INVALID_FIELD, "Property name length " + length + " exceeds maximum " + maxLength); + } + + if (pattern != null && !pattern.matcher(propertyName).matches()) { + return ValidatorUtils.createError(ValidatorUtils.buildFieldPath(fieldPath, propertyName), propertyName, + ValidationErrorType.INVALID_FIELD, "Property name does not match required pattern"); + } + + return null; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/RequiredFieldPresenceValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/RequiredFieldPresenceValidator.java new file mode 100644 index 000000000000..25771ebf81f5 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/RequiredFieldPresenceValidator.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.validation.RecordValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Set; + +/** + * Ensures that required JSON properties are actually present in the record, regardless of their value. + */ +public class RequiredFieldPresenceValidator implements RecordValidator { + private final Set requiredFields; + + public RequiredFieldPresenceValidator(final Set requiredFields) { + this.requiredFields = Set.copyOf(requiredFields); + } + + @Override + public Collection validate(final Record record, final RecordSchema schema, final String fieldPath) { + final Collection errors = new ArrayList<>(); + final Set rawFieldNames = record.getRawFieldNames(); + + for (final String requiredField : requiredFields) { + if (!rawFieldNames.contains(requiredField)) { + final String fullPath = ValidatorUtils.buildFieldPath(fieldPath, requiredField); + errors.add(ValidatorUtils.createError(fullPath, null, ValidationErrorType.MISSING_FIELD, "Field is required")); + } + } + + return errors; + } + + @Override + public String getDescription() { + return "Required field presence validator: " + requiredFields; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/StringLengthValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/StringLengthValidator.java new file mode 100644 index 000000000000..86cbb3449de0 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/StringLengthValidator.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; + +/** + * Validates {@code minLength} and {@code maxLength} constraints for string fields. + */ +public class StringLengthValidator implements FieldValidator { + private final Integer minLength; + private final Integer maxLength; + + public StringLengthValidator(final Integer minLength, final Integer maxLength) { + this.minLength = minLength; + this.maxLength = maxLength; + } + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + if (!(value instanceof CharSequence)) { + return ValidatorUtils.errorCollection(null); + } + + final int length = ((CharSequence) value).length(); + + if (minLength != null && length < minLength) { + final String explanation = String.format("String length must be at least %d", minLength); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + if (maxLength != null && length > maxLength) { + final String explanation = String.format("String length must be at most %d", maxLength); + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + final StringBuilder description = new StringBuilder("String length validator"); + if (minLength != null) { + description.append(", minLength=").append(minLength); + } + if (maxLength != null) { + description.append(", maxLength=").append(maxLength); + } + return description.toString(); + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/UniqueItemsValidator.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/UniqueItemsValidator.java new file mode 100644 index 000000000000..17273518a301 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/UniqueItemsValidator.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; +import java.util.HashSet; +import java.util.Set; + +/** + * Validates the JSON Schema {@code uniqueItems} constraint for arrays. + */ +public class UniqueItemsValidator implements FieldValidator { + + @Override + public Collection validate(final RecordField field, final String fieldPath, final Object value) { + final Object[] array = toArray(value); + if (array == null) { + return ValidatorUtils.errorCollection(null); + } + + final Set seen = new HashSet<>(); + for (final Object element : array) { + final String canonical = EnumValidator.canonicalize(element); + if (!seen.add(canonical)) { + final String explanation = "Array elements must be unique"; + final ValidationError error = ValidatorUtils.createError(fieldPath, value, ValidationErrorType.INVALID_FIELD, explanation); + return ValidatorUtils.errorCollection(error); + } + } + + return ValidatorUtils.errorCollection(null); + } + + @Override + public String getDescription() { + return "Unique items validator"; + } + + private Object[] toArray(final Object value) { + if (value instanceof Object[]) { + return (Object[]) value; + } + if (value instanceof java.util.Collection) { + return ((java.util.Collection) value).toArray(); + } + return null; + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ValidatorUtils.java b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ValidatorUtils.java new file mode 100644 index 000000000000..43056a851818 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/main/java/org/apache/nifi/json/schema/validation/ValidatorUtils.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.record.validation.DefaultValidationError; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.apache.nifi.serialization.record.validation.ValidationErrorType; + +import java.util.Collection; +import java.util.List; + +public final class ValidatorUtils { + private ValidatorUtils() { + } + + public static String buildFieldPath(final String basePath, final String fieldName) { + if (fieldName == null || fieldName.isEmpty()) { + return basePath == null ? "" : basePath; + } + + if (basePath == null || basePath.isEmpty()) { + return "/" + fieldName; + } + + if (basePath.endsWith("/")) { + return basePath + fieldName; + } + + return basePath + "/" + fieldName; + } + + public static Collection errorCollection(final ValidationError error) { + if (error == null) { + return List.of(); + } + return List.of(error); + } + + public static ValidationError createError(final String fieldPath, final Object value, final ValidationErrorType type, final String explanation) { + return DefaultValidationError.builder() + .fieldName(fieldPath) + .inputValue(value) + .type(type) + .explanation(explanation) + .build(); + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverterTest.java b/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverterTest.java new file mode 100644 index 000000000000..f1f4176fd0d9 --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/record/JsonSchemaToRecordSchemaConverterTest.java @@ -0,0 +1,514 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.record; + +import org.apache.nifi.json.schema.validation.AdditionalPropertiesValidator; +import org.apache.nifi.json.schema.validation.ArrayLengthValidator; +import org.apache.nifi.json.schema.validation.ConstValidator; +import org.apache.nifi.json.schema.validation.EnumValidator; +import org.apache.nifi.json.schema.validation.NumericRangeValidator; +import org.apache.nifi.json.schema.validation.RequiredFieldPresenceValidator; +import org.apache.nifi.json.schema.validation.StringLengthValidator; +import org.apache.nifi.json.schema.validation.UniqueItemsValidator; +import org.apache.nifi.schema.validation.SchemaValidationContext; +import org.apache.nifi.schema.validation.StandardSchemaValidator; +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; +import org.apache.nifi.serialization.record.type.ArrayDataType; +import org.apache.nifi.serialization.record.type.EnumDataType; +import org.apache.nifi.serialization.record.type.RecordDataType; +import org.apache.nifi.serialization.record.validation.SchemaValidationResult; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.fail; + +class JsonSchemaToRecordSchemaConverterTest { + private JsonSchemaToRecordSchemaConverter converter; + private RecordSchema recordSchema; + private RecordSchema metadataSchema; + + @BeforeEach + void setUp() { + converter = new JsonSchemaToRecordSchemaConverter(); + final String schemaText = """ + { + "$id": "urn:example:person", + "title": "Person", + "type": "object", + "additionalProperties": false, + "minProperties": 2, + "maxProperties": 8, + "required": ["id", "name", "metadata"], + "properties": { + "id": { + "type": ["string", "null"], + "minLength": 3 + }, + "name": { + "type": "string", + "enum": ["Alice", "Bob"] + }, + "age": { + "type": "integer", + "minimum": 0, + "exclusiveMaximum": 120 + }, + "score": { + "type": "number", + "exclusiveMinimum": 0, + "exclusiveMaximum": 1 + }, + "tags": { + "type": "array", + "items": { "type": "string" }, + "minItems": 1, + "uniqueItems": true + }, + "metadata": { + "type": "object", + "required": ["active"], + "properties": { + "active": { "type": "boolean" }, + "level": { + "type": ["integer", "null"], + "minimum": 1 + } + } + }, + "status": { + "const": "ACTIVE" + } + } + } + """; + recordSchema = converter.convert(schemaText); + final RecordField metadataField = recordSchema.getField("metadata") + .orElseThrow(() -> new AssertionError("metadata field missing")); + final DataType metadataDataType = metadataField.getDataType(); + metadataSchema = ((RecordDataType) metadataDataType).getChildSchema(); + } + + @Test + void testSchemaMetadataCaptured() { + assertEquals(Optional.of("Person"), recordSchema.getSchemaName()); + assertEquals(Optional.of("json-schema"), recordSchema.getSchemaFormat()); + assertEquals(Optional.of("urn:example:person"), recordSchema.getSchemaNamespace()); + + final RecordField nameField = recordSchema.getField("name").orElseThrow(); + assertEquals(RecordFieldType.ENUM, nameField.getDataType().getFieldType()); + assertTrue(nameField.getDataType() instanceof EnumDataType); + final EnumDataType enumDataType = (EnumDataType) nameField.getDataType(); + assertEquals(List.of("Alice", "Bob"), enumDataType.getEnums()); + assertTrue(nameField.getFieldValidators().stream().anyMatch(EnumValidator.class::isInstance)); + + final RecordField idField = recordSchema.getField("id").orElseThrow(); + assertTrue(idField.isNullable()); + assertTrue(idField.getFieldValidators().stream().anyMatch(StringLengthValidator.class::isInstance)); + + final RecordField ageField = recordSchema.getField("age").orElseThrow(); + assertTrue(ageField.getFieldValidators().stream().anyMatch(NumericRangeValidator.class::isInstance)); + + final RecordField scoreField = recordSchema.getField("score").orElseThrow(); + assertTrue(scoreField.getFieldValidators().stream().anyMatch(NumericRangeValidator.class::isInstance)); + + final RecordField tagsField = recordSchema.getField("tags").orElseThrow(); + assertEquals(RecordFieldType.ARRAY, tagsField.getDataType().getFieldType()); + final ArrayDataType arrayDataType = (ArrayDataType) tagsField.getDataType(); + assertEquals(RecordFieldType.STRING, arrayDataType.getElementType().getFieldType()); + assertTrue(tagsField.getFieldValidators().stream().anyMatch(ArrayLengthValidator.class::isInstance)); + assertTrue(tagsField.getFieldValidators().stream().anyMatch(UniqueItemsValidator.class::isInstance)); + + final RecordField statusField = recordSchema.getField("status").orElseThrow(); + assertTrue(statusField.getFieldValidators().stream().anyMatch(ConstValidator.class::isInstance)); + + assertEquals(2 + 1, recordSchema.getRecordValidators().size()); + assertTrue(recordSchema.getRecordValidators().stream().anyMatch(AdditionalPropertiesValidator.class::isInstance)); + assertTrue(recordSchema.getRecordValidators().stream().anyMatch(RequiredFieldPresenceValidator.class::isInstance)); + } + + @Test + void testValidationPassesForConformingRecord() { + final SchemaValidationContext context = new SchemaValidationContext(recordSchema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord metadataRecord = new MapRecord(metadataSchema, Map.of( + "active", true, + "level", 2 + )); + + final Map values = new HashMap<>(); + values.put("id", "abc"); + values.put("name", "Alice"); + values.put("age", 35); + values.put("score", 0.5D); + values.put("tags", new Object[]{"alpha"}); + values.put("metadata", metadataRecord); + values.put("status", "ACTIVE"); + + final MapRecord record = new MapRecord(recordSchema, values); + final SchemaValidationResult result = validator.validate(record); + assertTrue(result.isValid(), () -> "Expected valid record, found errors: " + result.getValidationErrors()); + } + + @Test + void testEnumViolationDetected() { + final SchemaValidationContext context = new SchemaValidationContext(recordSchema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord metadataRecord = new MapRecord(metadataSchema, Map.of("active", true)); + + final Map values = new HashMap<>(); + values.put("id", "abc"); + values.put("name", "Charlie"); + values.put("metadata", metadataRecord); + values.put("tags", new Object[]{"alpha"}); + values.put("status", "ACTIVE"); + + final MapRecord record = new MapRecord(recordSchema, values); + final SchemaValidationResult result = validator.validate(record); + assertFalse(result.isValid()); + assertTrue(result.getValidationErrors().stream().anyMatch(error -> error.getFieldName().map(name -> name.endsWith("name")).orElse(false))); + } + + @Test + void testAdditionalPropertyViolationDetected() { + final SchemaValidationContext context = new SchemaValidationContext(recordSchema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord metadataRecord = new MapRecord(metadataSchema, Map.of("active", true)); + + final Map values = new HashMap<>(); + values.put("id", "abc"); + values.put("name", "Alice"); + values.put("metadata", metadataRecord); + values.put("tags", new Object[]{"alpha"}); + values.put("status", "ACTIVE"); + values.put("extra", "value"); + + final MapRecord record = new MapRecord(recordSchema, values); + final SchemaValidationResult result = validator.validate(record); + assertFalse(result.isValid()); + assertTrue(result.getValidationErrors().stream().anyMatch(error -> error.getType() == org.apache.nifi.serialization.record.validation.ValidationErrorType.EXTRA_FIELD)); + } + + @Test + void testUniqueItemsViolationDetected() { + final SchemaValidationContext context = new SchemaValidationContext(recordSchema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord metadataRecord = new MapRecord(metadataSchema, Map.of("active", true)); + + final Map values = new HashMap<>(); + values.put("id", "abc"); + values.put("name", "Alice"); + values.put("metadata", metadataRecord); + values.put("tags", new Object[]{"dup", "dup"}); + values.put("status", "ACTIVE"); + + final MapRecord record = new MapRecord(recordSchema, values); + final SchemaValidationResult result = validator.validate(record); + assertFalse(result.isValid()); + assertTrue(result.getValidationErrors().stream().anyMatch(error -> error.getFieldName().map(name -> name.contains("tags")).orElse(false))); + } + + @Test + void testRequiredNullableFieldPresenceValidatorTriggersWhenMissing() { + final SchemaValidationContext context = new SchemaValidationContext(recordSchema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord metadataRecord = new MapRecord(metadataSchema, Map.of("active", true)); + + final Map values = new HashMap<>(); + values.put("name", "Alice"); + values.put("metadata", metadataRecord); + values.put("tags", new Object[]{"alpha"}); + values.put("status", "ACTIVE"); + + final MapRecord record = new MapRecord(recordSchema, values); + final SchemaValidationResult result = validator.validate(record); + assertFalse(result.isValid()); + assertTrue(result.getValidationErrors().stream().anyMatch(error -> error.getType() == org.apache.nifi.serialization.record.validation.ValidationErrorType.MISSING_FIELD)); + } + + @Test + void testDateAndTimeValuesValidated() { + final String schemaText = """ + { + "type": "object", + "properties": { + "releaseDate": { + "type": "string", + "format": "date" + }, + "releaseTime": { + "type": "string", + "format": "time" + }, + "lastUpdated": { + "type": "string", + "format": "date-time" + } + } + } + """; + + final RecordSchema schema = converter.convert(schemaText); + final RecordField dateField = schema.getField("releaseDate").orElseThrow(); + assertEquals(RecordFieldType.DATE, dateField.getDataType().getFieldType()); + + final RecordField timeField = schema.getField("releaseTime").orElseThrow(); + assertEquals(RecordFieldType.TIME, timeField.getDataType().getFieldType()); + + final RecordField timestampField = schema.getField("lastUpdated").orElseThrow(); + assertEquals(RecordFieldType.TIMESTAMP, timestampField.getDataType().getFieldType()); + + final SchemaValidationContext context = new SchemaValidationContext(schema, false, false); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord validRecord = new MapRecord(schema, Map.of( + "releaseDate", "2024-01-15", + "releaseTime", "13:45:00", + "lastUpdated", "2024-01-15T13:45:00Z" + )); + assertTrue(validator.validate(validRecord).isValid()); + + final MapRecord invalidRecord = new MapRecord(schema, Map.of( + "releaseDate", "2024-13-01", + "releaseTime", "25:61:00", + "lastUpdated", "not-a-timestamp" + )); + assertFalse(validator.validate(invalidRecord).isValid()); + } + + @Test + void testPatternPropertiesValidation() { + final String schemaText = """ + { + "type": "object", + "additionalProperties": false, + "properties": { + "fixed": { "type": "string" } + }, + "patternProperties": { + "^x-": { + "type": "string", + "minLength": 2 + }, + "^meta-": { + "type": "object", + "required": ["enabled"], + "properties": { + "enabled": { "type": "boolean" } + } + } + } + } + """; + + final RecordSchema schema = converter.convert(schemaText); + final SchemaValidationContext context = new SchemaValidationContext(schema, true, false); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final Map validValues = new HashMap<>(); + validValues.put("fixed", "value"); + validValues.put("x-ok", "ab"); + validValues.put("meta-config", Map.of("enabled", true)); + + final MapRecord validRecord = new MapRecord(schema, validValues); + final SchemaValidationResult validResult = validator.validate(validRecord); + final boolean valid = validResult.isValid(); + if (!valid) { + fail("Unexpected property name validation errors: " + validResult.getValidationErrors()); + } + + final Map invalidValues = new HashMap<>(); + invalidValues.put("fixed", "value"); + invalidValues.put("x-bad", "a"); + invalidValues.put("meta-config", Map.of()); + invalidValues.put("other", "not allowed"); + + final MapRecord invalidRecord = new MapRecord(schema, invalidValues); + final SchemaValidationResult result = validator.validate(invalidRecord); + assertFalse(result.isValid()); + assertEquals(3, result.getValidationErrors().size()); + + final List explanations = result.getValidationErrors().stream() + .map(ValidationError::getExplanation) + .toList(); + + assertTrue(explanations.stream().anyMatch(msg -> msg.contains("String length"))); + assertTrue(explanations.stream().anyMatch(msg -> msg.contains("Field is required"))); + assertTrue(explanations.stream().anyMatch(msg -> msg.contains("not defined"))); + } + + @Test + void testPropertyNamesValidation() { + final JsonSchemaToRecordSchemaConverter localConverter = new JsonSchemaToRecordSchemaConverter(); + final String schemaText = """ + { + "$defs": { + "LowerAlpha": { + "type": "string", + "pattern": "^[a-z]+$", + "minLength": 3 + } + }, + "type": "object", + "propertyNames": { "$ref": "#/$defs/LowerAlpha" }, + "additionalProperties": true + } + """; + + final RecordSchema schema = localConverter.convert(schemaText); + final SchemaValidationContext context = new SchemaValidationContext(schema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord validRecord = new MapRecord(schema, Map.of( + "alpha", 1, + "beta", "value" + )); + final SchemaValidationResult validResult = validator.validate(validRecord); + if (!validResult.isValid()) { + fail("Unexpected property name validation errors: " + validResult.getValidationErrors()); + } + + final MapRecord invalidRecord = new MapRecord(schema, Map.of( + "A", 1, + "beta", "value" + )); + final SchemaValidationResult invalidResult = validator.validate(invalidRecord); + assertFalse(invalidResult.isValid()); + assertTrue(invalidResult.getValidationErrors().stream() + .anyMatch(error -> error.getExplanation().contains("pattern") || error.getExplanation().contains("length"))); + } + + @Test + void testReferencedObjectSchemaConverted() { + final JsonSchemaToRecordSchemaConverter localConverter = new JsonSchemaToRecordSchemaConverter(); + final String schemaText = """ + { + "$defs": { + "Address": { + "type": "object", + "required": ["city"], + "properties": { + "city": { "type": "string" }, + "zip": { "type": "string", "minLength": 5 } + } + } + }, + "type": "object", + "properties": { + "address": { "$ref": "#/$defs/Address" } + }, + "required": ["address"] + } + """; + + final RecordSchema schema = localConverter.convert(schemaText); + final RecordField addressField = schema.getField("address").orElseThrow(); + assertEquals(RecordFieldType.RECORD, addressField.getDataType().getFieldType()); + + final RecordSchema addressSchema = ((RecordDataType) addressField.getDataType()).getChildSchema(); + final SchemaValidationContext context = new SchemaValidationContext(schema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final MapRecord validAddress = new MapRecord(addressSchema, Map.of( + "city", "Paris", + "zip", "75000" + )); + final MapRecord validRecord = new MapRecord(schema, Map.of("address", validAddress)); + assertTrue(validator.validate(validRecord).isValid()); + + final MapRecord invalidAddress = new MapRecord(addressSchema, Map.of("zip", "123")); + final MapRecord invalidRecord = new MapRecord(schema, Map.of("address", invalidAddress)); + final SchemaValidationResult invalidResult = validator.validate(invalidRecord); + assertFalse(invalidResult.isValid()); + assertTrue(invalidResult.getValidationErrors().stream().anyMatch(error -> error.getFieldName().map(name -> name.contains("city")).orElse(false))); + } + + @Test + void testPatternPropertiesReferenceValidation() { + final JsonSchemaToRecordSchemaConverter localConverter = new JsonSchemaToRecordSchemaConverter(); + final String schemaText = """ + { + "$defs": { + "Header": { + "type": "string", + "minLength": 2 + } + }, + "type": "object", + "additionalProperties": false, + "properties": { + "fixed": { "type": "string" } + }, + "patternProperties": { + "^x-": { "$ref": "#/$defs/Header" } + }, + "required": ["fixed"] + } + """; + + final RecordSchema schema = localConverter.convert(schemaText); + final SchemaValidationContext context = new SchemaValidationContext(schema, true, true); + final StandardSchemaValidator validator = new StandardSchemaValidator(context); + + final Map validValues = new HashMap<>(); + validValues.put("fixed", "value"); + validValues.put("x-trace", "ab"); + final SchemaValidationResult validResult = validator.validate(new MapRecord(schema, validValues)); + assertTrue(validResult.isValid(), () -> "Expected valid record, found errors: " + validResult.getValidationErrors()); + + final Map invalidValues = new HashMap<>(); + invalidValues.put("fixed", "value"); + invalidValues.put("x-trace", "a"); + final SchemaValidationResult invalidResult = validator.validate(new MapRecord(schema, invalidValues)); + assertFalse(invalidResult.isValid()); + assertTrue(invalidResult.getValidationErrors().stream().anyMatch(error -> error.getFieldName().map("/x-trace"::equals).orElse(false))); + } + + @Test + void testExternalReferenceNotSupported() { + final JsonSchemaToRecordSchemaConverter localConverter = new JsonSchemaToRecordSchemaConverter(); + final String schemaText = """ + { + "type": "object", + "properties": { + "ref": { "$ref": "http://example.com/external.json#/Some" } + } + } + """; + + final IllegalArgumentException exception = assertThrows(IllegalArgumentException.class, () -> localConverter.convert(schemaText)); + assertTrue(exception.getMessage().contains("External JSON Schema references")); + } +} diff --git a/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidatorTest.java b/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidatorTest.java new file mode 100644 index 000000000000..f8a47a706f7e --- /dev/null +++ b/nifi-commons/nifi-json-schema-shared/src/test/java/org/apache/nifi/json/schema/validation/PatternPropertiesValidatorTest.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.json.schema.validation; + +import org.apache.nifi.serialization.SimpleRecordSchema; +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; +import org.apache.nifi.serialization.record.validation.ValidationError; +import org.junit.jupiter.api.Test; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +class PatternPropertiesValidatorTest { + + @Test + void testDynamicFieldsValidatedAgainstTemplate() { + final RecordField templateField = new RecordField( + "pattern", + RecordFieldType.STRING.getDataType(), + null, + Collections.emptySet(), + true, + List.of(new StringLengthValidator(2, null)) + ); + + final PatternPropertiesValidator validator = new PatternPropertiesValidator(List.of( + new PatternPropertiesValidator.PatternPropertyDefinition(Pattern.compile("^x-"), templateField) + )); + + final RecordField fixedField = new RecordField("fixed", RecordFieldType.STRING.getDataType()); + final RecordSchema schema = new SimpleRecordSchema(List.of(fixedField)); + final MapRecord record = new MapRecord(schema, Map.of( + "fixed", "value", + "x-good", "ok", + "x-bad", "a" + )); + + final Collection validationErrors = validator.validate(record, schema, ""); + + assertEquals(1, validationErrors.size()); + final ValidationError error = validationErrors.iterator().next(); + assertEquals("/x-bad", error.getFieldName().orElse(null)); + assertTrue(error.getExplanation().contains("String length")); + } + + @Test + void testExplicitFieldsAreNotRevalidatedByPattern() { + final RecordField templateField = new RecordField( + "pattern", + RecordFieldType.STRING.getDataType(), + null, + Collections.emptySet(), + true, + List.of(new StringLengthValidator(2, null)) + ); + + final PatternPropertiesValidator validator = new PatternPropertiesValidator(List.of( + new PatternPropertiesValidator.PatternPropertyDefinition(Pattern.compile("^x-"), templateField) + )); + + final RecordField fixedField = new RecordField("fixed", RecordFieldType.STRING.getDataType()); + final RecordField explicitField = new RecordField("x-fixed", RecordFieldType.STRING.getDataType()); + final RecordSchema schema = new SimpleRecordSchema(List.of(fixedField, explicitField)); + final MapRecord record = new MapRecord(schema, Map.of( + "fixed", "value", + "x-fixed", "a", + "x-dynamic", "ab" + )); + + final Collection validationErrors = validator.validate(record, schema, ""); + + assertTrue(validationErrors.isEmpty(), "Expected no validation errors for explicit field"); + } +} diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java index b88180485f0d..26d856cc021b 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java @@ -22,6 +22,7 @@ import org.apache.nifi.serialization.record.RecordFieldRemovalPath; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.SchemaIdentifier; +import org.apache.nifi.serialization.record.validation.RecordValidator; import java.util.ArrayList; import java.util.HashMap; @@ -41,6 +42,7 @@ public class SimpleRecordSchema implements RecordSchema { private String schemaNamespace; private volatile int hashCode; private volatile Boolean recursive; + private List recordValidators = List.of(); public SimpleRecordSchema(final List fields) { this(fields, null, null, false, SchemaIdentifier.EMPTY); @@ -181,12 +183,12 @@ public boolean equals(final Object obj) { && getSchemaName().isPresent() && getSchemaName().equals(other.getSchemaName())) { return true; } else { - return fields.equals(other.getFields()); + return fields.equals(other.getFields()) && getRecordValidators().equals(other.getRecordValidators()); } } else if (thisIsRecursive || otherIsRecursive) { return false; } else { - return fields.equals(other.getFields()); + return fields.equals(other.getFields()) && getRecordValidators().equals(other.getRecordValidators()); } } @@ -201,7 +203,7 @@ public int hashCode() { } private int calculateHashCode() { - return 143 + 3 * fields.hashCode(); + return 143 + 3 * fields.hashCode() + 7 * recordValidators.hashCode(); } @@ -331,6 +333,20 @@ private void resetFields(final List updatedFields) { } + public void setRecordValidators(final List recordValidators) { + if (recordValidators == null || recordValidators.isEmpty()) { + this.recordValidators = List.of(); + } else { + this.recordValidators = List.copyOf(recordValidators); + } + this.hashCode = 0; + } + + @Override + public List getRecordValidators() { + return recordValidators; + } + @Override public boolean isRecursive() { final Boolean recursiveFlag = this.recursive; diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java index 125a44dbcd36..3f7276ad1075 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordField.java @@ -20,9 +20,12 @@ import org.apache.nifi.serialization.record.util.DataTypeUtils; import java.util.Collections; +import java.util.List; import java.util.Objects; import java.util.Set; +import org.apache.nifi.serialization.record.validation.FieldValidator; + public class RecordField { public static final boolean DEFAULT_NULLABLE = true; @@ -31,6 +34,7 @@ public class RecordField { private final Set aliases; private final Object defaultValue; private final boolean nullable; + private final List fieldValidators; public RecordField(final String fieldName, final DataType dataType) { this(fieldName, dataType, null, Collections.emptySet(), DEFAULT_NULLABLE); @@ -61,6 +65,11 @@ public RecordField(final String fieldName, final DataType dataType, final Object } public RecordField(final String fieldName, final DataType dataType, final Object defaultValue, final Set aliases, final boolean nullable) { + this(fieldName, dataType, defaultValue, aliases, nullable, List.of()); + } + + public RecordField(final String fieldName, final DataType dataType, final Object defaultValue, final Set aliases, final boolean nullable, + final List fieldValidators) { if (defaultValue != null && !DataTypeUtils.isCompatibleDataType(defaultValue, dataType)) { throw new IllegalArgumentException("Cannot set the default value for field [" + fieldName + "] to [" + defaultValue + "] because that is not a valid value for Data Type [" + dataType + "]"); @@ -79,6 +88,8 @@ public RecordField(final String fieldName, final DataType dataType, final Object this.defaultValue = defaultValue; this.nullable = nullable; + Objects.requireNonNull(fieldValidators, "Field validators cannot be null"); + this.fieldValidators = fieldValidators.isEmpty() ? List.of() : List.copyOf(fieldValidators); } public String getFieldName() { @@ -101,6 +112,10 @@ public boolean isNullable() { return nullable; } + public List getFieldValidators() { + return fieldValidators; + } + @Override public int hashCode() { final int prime = 31; @@ -110,6 +125,7 @@ public int hashCode() { result = prime * result + aliases.hashCode(); result = prime * result + ((defaultValue == null) ? 0 : defaultValue.hashCode()); result = prime * result + Boolean.hashCode(nullable); + result = prime * result + fieldValidators.hashCode(); return result; } @@ -128,11 +144,13 @@ public boolean equals(Object obj) { RecordField other = (RecordField) obj; return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName()) && aliases.equals(other.getAliases()) && Objects.equals(defaultValue, other.defaultValue) - && nullable == other.nullable; + && nullable == other.nullable + && fieldValidators.equals(other.fieldValidators); } @Override public String toString() { - return "RecordField[name=" + fieldName + ", dataType=" + dataType + (aliases.isEmpty() ? "" : ", aliases=" + aliases) + ", nullable=" + nullable + "]"; + return "RecordField[name=" + fieldName + ", dataType=" + dataType + (aliases.isEmpty() ? "" : ", aliases=" + aliases) + + ", nullable=" + nullable + (fieldValidators.isEmpty() ? "" : ", validators=" + fieldValidators.size()) + "]"; } } diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java index c085fde4216b..226f5cba63d1 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java @@ -20,6 +20,8 @@ import java.util.List; import java.util.Optional; +import org.apache.nifi.serialization.record.validation.RecordValidator; + public interface RecordSchema { /** * @return the list of fields that are present in the schema @@ -112,6 +114,14 @@ public interface RecordSchema { */ boolean isRecursive(); + /** + * @return validators that should be applied to records adhering to this schema. The default implementation returns an empty list to + * maintain backwards compatibility for existing schema implementations that do not expose custom validators. + */ + default List getRecordValidators() { + return List.of(); + } + /** * @param schemas the list of schemas to check whether the current schema is contained within * @return true if the current schema is present within the list of schemas (object reference equality), false otherwise diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateFieldConverter.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateFieldConverter.java index 63f2633998fd..b105abf920e2 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateFieldConverter.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateFieldConverter.java @@ -24,12 +24,19 @@ import java.time.ZoneId; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; +import java.time.format.ResolverStyle; import java.util.Optional; /** * Convert Object to java.time.LocalDate using instanceof evaluation and optional format pattern for DateTimeFormatter */ class ObjectLocalDateFieldConverter implements FieldConverter { + private static final DateTimeFormatter[] DEFAULT_DATE_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_LOCAL_DATE.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_OFFSET_DATE.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_DATE.withResolverStyle(ResolverStyle.STRICT) + }; + /** * Convert Object field to java.sql.Timestamp using optional format supported in DateTimeFormatter * @@ -72,15 +79,23 @@ public LocalDate convertField(final Object field, final Optional pattern } catch (final DateTimeParseException e) { throw new FieldConversionException(LocalDate.class, field, name, e); } - } else { + } + + for (final DateTimeFormatter formatter : DEFAULT_DATE_FORMATTERS) { try { - final long number = Long.parseLong(string); - final Instant instant = Instant.ofEpochMilli(number); - return ofInstant(instant); - } catch (final NumberFormatException e) { - throw new FieldConversionException(LocalDate.class, field, name, e); + return LocalDate.parse(string, formatter); + } catch (final DateTimeParseException e) { + continue; } } + + try { + final long number = Long.parseLong(string); + final Instant instant = Instant.ofEpochMilli(number); + return ofInstant(instant); + } catch (final NumberFormatException e) { + throw new FieldConversionException(LocalDate.class, field, name, e); + } } default -> { } diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateTimeFieldConverter.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateTimeFieldConverter.java index b5573cc9e702..40c7e67c902b 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateTimeFieldConverter.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalDateTimeFieldConverter.java @@ -25,6 +25,7 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; +import java.time.format.ResolverStyle; import java.time.temporal.TemporalAccessor; import java.time.temporal.TemporalQueries; import java.time.temporal.TemporalQuery; @@ -38,6 +39,11 @@ class ObjectLocalDateTimeFieldConverter implements FieldConverter LOCAL_DATE_TIME_TEMPORAL_QUERY = new LocalDateTimeQuery(); + private static final DateTimeFormatter[] DEFAULT_DATE_TIME_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_OFFSET_DATE_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_LOCAL_DATE_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_INSTANT.withResolverStyle(ResolverStyle.STRICT) + }; /** * Convert Object field to java.sql.Timestamp using optional format supported in DateTimeFormatter @@ -86,9 +92,17 @@ public LocalDateTime convertField(final Object field, final Optional pat } catch (final DateTimeParseException e) { return tryParseAsNumber(string, name); } - } else { - return tryParseAsNumber(string, name); } + + for (final DateTimeFormatter formatter : DEFAULT_DATE_TIME_FORMATTERS) { + try { + return formatter.parse(string, LOCAL_DATE_TIME_TEMPORAL_QUERY); + } catch (final DateTimeParseException e) { + continue; + } + } + + return tryParseAsNumber(string, name); } default -> { } diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalTimeFieldConverter.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalTimeFieldConverter.java index 178993ed9317..461afcfd2049 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalTimeFieldConverter.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/field/ObjectLocalTimeFieldConverter.java @@ -25,6 +25,7 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; +import java.time.format.ResolverStyle; import java.util.Date; import java.util.Optional; @@ -32,6 +33,12 @@ * Convert Object to java.time.LocalTime using instanceof evaluation and optional format pattern for DateTimeFormatter */ class ObjectLocalTimeFieldConverter implements FieldConverter { + private static final DateTimeFormatter[] DEFAULT_TIME_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_LOCAL_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_OFFSET_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_TIME.withResolverStyle(ResolverStyle.STRICT) + }; + /** * Convert Object field to java.time.LocalTime using optional format supported in DateTimeFormatter * @@ -76,15 +83,23 @@ public LocalTime convertField(final Object field, final Optional pattern } catch (final DateTimeParseException e) { throw new FieldConversionException(LocalTime.class, field, name, e); } - } else { + } + + for (final DateTimeFormatter formatter : DEFAULT_TIME_FORMATTERS) { try { - final long number = Long.parseLong(string); - final Instant instant = Instant.ofEpochMilli(number); - return ofInstant(instant); - } catch (final NumberFormatException e) { - throw new FieldConversionException(LocalTime.class, field, name, e); + return LocalTime.parse(string, formatter); + } catch (final DateTimeParseException e) { + continue; } } + + try { + final long number = Long.parseLong(string); + final Instant instant = Instant.ofEpochMilli(number); + return ofInstant(instant); + } catch (final NumberFormatException e) { + throw new FieldConversionException(LocalTime.class, field, name, e); + } } default -> { } diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index 0e7b0c9b91f2..aae1c110e94f 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -57,6 +57,7 @@ import java.time.ZonedDateTime; import java.time.format.DateTimeFormatter; import java.time.format.DateTimeParseException; +import java.time.format.ResolverStyle; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -313,13 +314,13 @@ public static boolean isCompatibleDataType(final Object value, final DataType da case TIME -> isTimeTypeCompatible(value, dataType.getFormat()); case TIMESTAMP -> isTimestampTypeCompatible(value, dataType.getFormat()); case STRING -> isStringTypeCompatible(value); + case UUID -> isUUIDTypeCompatible(value); case ENUM -> isEnumTypeCompatible(value, (EnumDataType) dataType); case MAP -> isMapTypeCompatible(value); case CHOICE -> { final DataType chosenDataType = chooseDataType(value, (ChoiceDataType) dataType); yield chosenDataType != null; } - default -> false; }; } @@ -716,12 +717,12 @@ private static boolean isRecordTypeCompatible(RecordSchema schema, Object value, childValue = ((Map) value).get(childField.getFieldName()); } - if (childValue == null && !childField.isNullable()) { - logger.debug("Value is not compatible with schema because field {} has a null value, which is not allowed in the schema", childField.getFieldName()); - return false; - } if (childValue == null) { - continue; // consider compatible + if (strict && !childField.isNullable()) { + logger.debug("Value is not compatible with schema because field {} has a null value, which is not allowed in the schema under strict validation", childField.getFieldName()); + return false; + } + continue; // consider compatible and rely on downstream validation } if (!isCompatibleDataType(childValue, childField.getDataType(), strict)) { @@ -1143,7 +1144,29 @@ public static Date convertDateToUTC(Date dateLocalTZ) { return new Date(zdtUTC.toInstant().toEpochMilli()); } + private static final DateTimeFormatter[] DEFAULT_DATE_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_LOCAL_DATE.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_OFFSET_DATE.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_DATE.withResolverStyle(ResolverStyle.STRICT) + }; + + private static final DateTimeFormatter[] DEFAULT_TIME_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_LOCAL_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_OFFSET_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_TIME.withResolverStyle(ResolverStyle.STRICT) + }; + + private static final DateTimeFormatter[] DEFAULT_DATETIME_FORMATTERS = new DateTimeFormatter[]{ + DateTimeFormatter.ISO_OFFSET_DATE_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_LOCAL_DATE_TIME.withResolverStyle(ResolverStyle.STRICT), + DateTimeFormatter.ISO_INSTANT.withResolverStyle(ResolverStyle.STRICT) + }; + public static boolean isDateTypeCompatible(final Object value, final String format) { + return isDateTypeCompatible(value, format, true); + } + + public static boolean isDateTypeCompatible(final Object value, final String format, final boolean allowDefaultFormats) { if (value == null) { return false; } @@ -1152,17 +1175,87 @@ public static boolean isDateTypeCompatible(final Object value, final String form return true; } - if (value instanceof String) { - if (format == null) { - return isInteger((String) value); + if (value instanceof String stringValue) { + final String trimmed = stringValue.trim(); + if (trimmed.isEmpty()) { + return false; } - try { - DateTimeFormatter.ofPattern(format).parse(value.toString()); - return true; - } catch (final DateTimeParseException e) { + if (format != null && !format.isBlank()) { + return isParsable(trimmed, DateTimeFormatter.ofPattern(format).withResolverStyle(ResolverStyle.STRICT)); + } + + if (!allowDefaultFormats) { + return isInteger(trimmed); + } + + return isParsable(trimmed, DEFAULT_DATE_FORMATTERS); + } + + return false; + } + + public static boolean isTimeTypeCompatible(final Object value, final String format) { + return isTimeTypeCompatible(value, format, true); + } + + public static boolean isTimeTypeCompatible(final Object value, final String format, final boolean allowDefaultFormats) { + if (value == null) { + return false; + } + + if (value instanceof java.util.Date || value instanceof Number) { + return true; + } + + if (value instanceof String stringValue) { + final String trimmed = stringValue.trim(); + if (trimmed.isEmpty()) { return false; } + + if (format != null && !format.isBlank()) { + return isParsable(trimmed, DateTimeFormatter.ofPattern(format).withResolverStyle(ResolverStyle.STRICT)); + } + + if (!allowDefaultFormats) { + return isInteger(trimmed); + } + + return isParsable(trimmed, DEFAULT_TIME_FORMATTERS); + } + + return false; + } + + public static boolean isTimestampTypeCompatible(final Object value, final String format) { + return isTimestampTypeCompatible(value, format, true); + } + + public static boolean isTimestampTypeCompatible(final Object value, final String format, final boolean allowDefaultFormats) { + if (value == null) { + return false; + } + + if (value instanceof java.util.Date || value instanceof Number) { + return true; + } + + if (value instanceof String stringValue) { + final String trimmed = stringValue.trim(); + if (trimmed.isEmpty()) { + return false; + } + + if (format != null && !format.isBlank()) { + return isParsable(trimmed, DateTimeFormatter.ofPattern(format).withResolverStyle(ResolverStyle.STRICT)); + } + + if (!allowDefaultFormats) { + return isInteger(trimmed); + } + + return isParsable(trimmed, DEFAULT_DATETIME_FORMATTERS); } return false; @@ -1182,12 +1275,50 @@ private static boolean isInteger(final String value) { return true; } - public static boolean isTimeTypeCompatible(final Object value, final String format) { - return isDateTypeCompatible(value, format); + private static boolean isParsable(final String value, final DateTimeFormatter... formatters) { + for (final DateTimeFormatter formatter : formatters) { + try { + formatter.parse(value); + return true; + } catch (final DateTimeParseException e) { + continue; + } + } + return false; } - public static boolean isTimestampTypeCompatible(final Object value, final String format) { - return isDateTypeCompatible(value, format); + public static boolean isUUIDTypeCompatible(final Object value) { + if (value == null) { + return false; + } + + if (value instanceof UUID) { + return true; + } + + if (value instanceof byte[] bytes) { + return bytes.length == 16; + } + + if (value instanceof Byte[] array) { + return array.length == 16; + } + + if (value instanceof String stringValue) { + final String trimmed = stringValue.trim(); + if (trimmed.isEmpty()) { + return false; + } + + try { + UUID.fromString(trimmed); + return true; + } catch (final IllegalArgumentException e) { + return false; + } + } + + return false; } public static BigInteger toBigInt(final Object value, final String fieldName) { diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/DefaultValidationError.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/DefaultValidationError.java new file mode 100644 index 000000000000..0977400ed9c0 --- /dev/null +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/DefaultValidationError.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.serialization.record.validation; + +import java.util.Objects; +import java.util.Optional; + +/** + * Basic implementation of {@link ValidationError} that can be used by validators in modules that + * cannot depend on higher level utility classes. Instances are immutable and thread-safe. + */ +public class DefaultValidationError implements ValidationError { + private final Optional fieldName; + private final Optional inputValue; + private final String explanation; + private final ValidationErrorType type; + + private DefaultValidationError(final Builder builder) { + this.fieldName = Optional.ofNullable(builder.fieldName); + this.inputValue = Optional.ofNullable(builder.inputValue); + this.explanation = Objects.requireNonNull(builder.explanation, "Explanation is required"); + this.type = Objects.requireNonNull(builder.type, "Validation error type is required"); + } + + @Override + public Optional getFieldName() { + return fieldName; + } + + @Override + public Optional getInputValue() { + return inputValue; + } + + @Override + public String getExplanation() { + return explanation; + } + + @Override + public ValidationErrorType getType() { + return type; + } + + /** + * Creates a builder for constructing immutable {@link DefaultValidationError} instances. + * + * @return builder instance + */ + public static Builder builder() { + return new Builder(); + } + + public static final class Builder { + private String fieldName; + private Object inputValue; + private String explanation; + private ValidationErrorType type = ValidationErrorType.INVALID_FIELD; + + private Builder() { + } + + public Builder fieldName(final String fieldName) { + this.fieldName = fieldName; + return this; + } + + public Builder inputValue(final Object inputValue) { + this.inputValue = inputValue; + return this; + } + + public Builder explanation(final String explanation) { + this.explanation = explanation; + return this; + } + + public Builder type(final ValidationErrorType type) { + this.type = type; + return this; + } + + public DefaultValidationError build() { + return new DefaultValidationError(this); + } + } +} diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/FieldValidator.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/FieldValidator.java new file mode 100644 index 000000000000..241e253d03e5 --- /dev/null +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/FieldValidator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.serialization.record.validation; + +import org.apache.nifi.serialization.record.RecordField; + +import java.util.Collection; + +/** + * Provides validation for an individual {@link RecordField}. Field Validators are expected to be immutable and thread-safe. + */ +public interface FieldValidator { + + /** + * Validates the provided value for the given field. + * + * @param field the record field being validated + * @param fieldPath the path of the field being validated (used for clear diagnostics) + * @param value the value of the field for the record currently being validated + * @return a collection of validation errors. The collection must be empty when the value is valid. + */ + Collection validate(RecordField field, String fieldPath, Object value); + + /** + * @return a short human readable description of what the validator enforces + */ + String getDescription(); +} diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/RecordValidator.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/RecordValidator.java new file mode 100644 index 000000000000..c39b952e18e9 --- /dev/null +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/validation/RecordValidator.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.serialization.record.validation; + +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; + +import java.util.Collection; + +/** + * Provides validation logic for an entire {@link Record} instance against a {@link RecordSchema}. + * Record Validators are expected to be immutable and thread-safe. + */ +public interface RecordValidator { + + /** + * Validates the provided record. + * + * @param record the record instance to validate + * @param schema the schema the record should adhere to + * @param fieldPath the path within the overall document that identifies the record (root records use the empty string) + * @return a collection of validation errors. The collection must be empty when the record is valid. + */ + Collection validate(Record record, RecordSchema schema, String fieldPath); + + /** + * @return a short human readable description of what the validator enforces + */ + String getDescription(); +} diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java index 7fede6c955cd..76b50af724ab 100644 --- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java @@ -1223,4 +1223,69 @@ void testNumberParsingWhereStringBlank() { assertNull(DataTypeUtils.toLong("", fieldName)); assertNull(DataTypeUtils.toShort("", fieldName)); } + + @Test + void testIsoDateCompatibilityWithoutExplicitFormat() { + assertTrue(DataTypeUtils.isDateTypeCompatible("2024-01-15", null)); + assertTrue(DataTypeUtils.isDateTypeCompatible(" 2024-01-15 ", null)); + assertTrue(DataTypeUtils.isDateTypeCompatible("2024-01-15Z", null)); + + assertFalse(DataTypeUtils.isDateTypeCompatible("15/01/2024", null)); + assertFalse(DataTypeUtils.isDateTypeCompatible("", null)); + } + + @Test + void testIsoTimeCompatibilityWithoutExplicitFormat() { + assertTrue(DataTypeUtils.isTimeTypeCompatible("13:45:00", null)); + assertTrue(DataTypeUtils.isTimeTypeCompatible("13:45:00Z", null)); + assertTrue(DataTypeUtils.isTimeTypeCompatible(" 13:45:00.123 ", null)); + + assertFalse(DataTypeUtils.isTimeTypeCompatible("25:61", null)); + assertFalse(DataTypeUtils.isTimeTypeCompatible("", null)); + } + + @Test + void testIsoTimestampCompatibilityWithoutExplicitFormat() { + assertTrue(DataTypeUtils.isTimestampTypeCompatible("2024-01-15T13:45:00Z", null)); + assertTrue(DataTypeUtils.isTimestampTypeCompatible("2024-01-15T13:45:00", null)); + assertTrue(DataTypeUtils.isTimestampTypeCompatible(" 2024-01-15T13:45:00+01:00 ", null)); + + assertFalse(DataTypeUtils.isTimestampTypeCompatible("not-a-timestamp", null)); + assertFalse(DataTypeUtils.isTimestampTypeCompatible("", null)); + } + + @Test + void testLegacyEpochCompatibilityWhenDefaultFormatsDisabled() { + // The three-argument overload keeps AbstractCSVRecordReader's non-coerce path aligned with legacy behaviour (see + // nifi-extension-bundles/.../csv/AbstractCSVRecordReader.java:122) by rejecting ISO strings when no explicit format + // is provided while still allowing epoch-based values. + assertFalse(DataTypeUtils.isDateTypeCompatible("2024-01-15", null, false)); + assertFalse(DataTypeUtils.isTimeTypeCompatible("01:02:03", null, false)); + assertFalse(DataTypeUtils.isTimestampTypeCompatible("2024-01-15T01:02:03Z", null, false)); + + assertTrue(DataTypeUtils.isDateTypeCompatible(String.valueOf(1700000000000L), null, false)); + assertTrue(DataTypeUtils.isTimeTypeCompatible(String.valueOf(1700000000000L), null, false)); + assertTrue(DataTypeUtils.isTimestampTypeCompatible(String.valueOf(1700000000000L), null, false)); + } + + @Test + void testUuidCompatibilityAcrossSupportedRepresentations() { + final UUID uuid = UUID.randomUUID(); + assertTrue(DataTypeUtils.isUUIDTypeCompatible(uuid)); + assertTrue(DataTypeUtils.isUUIDTypeCompatible(uuid.toString())); + assertTrue(DataTypeUtils.isUUIDTypeCompatible(" " + uuid + " ")); + + final byte[] uuidBytes = new byte[16]; + Arrays.fill(uuidBytes, (byte) 1); + assertTrue(DataTypeUtils.isUUIDTypeCompatible(uuidBytes)); + + final Byte[] boxedBytes = new Byte[16]; + Arrays.fill(boxedBytes, (byte) 2); + assertTrue(DataTypeUtils.isUUIDTypeCompatible(boxedBytes)); + + assertFalse(DataTypeUtils.isUUIDTypeCompatible(new byte[15])); + assertFalse(DataTypeUtils.isUUIDTypeCompatible(new Byte[15])); + assertFalse(DataTypeUtils.isUUIDTypeCompatible("not-a-uuid")); + assertFalse(DataTypeUtils.isUUIDTypeCompatible("")); + } } diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateFieldConverter.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateFieldConverter.java new file mode 100644 index 000000000000..f760b473cfb1 --- /dev/null +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateFieldConverter.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.serialization.record.field; + +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +class TestObjectLocalDateFieldConverter { + private static final ObjectLocalDateFieldConverter CONVERTER = new ObjectLocalDateFieldConverter(); + + @Test + void testDefaultFormatterIterationFallsBackToNumericParse() { + // Covers the default formatter loop in ObjectLocalDateFieldConverter.convertField which now uses 'continue' + // (see ObjectLocalDateFieldConverter.java:84) to keep iterating before attempting the numeric fallback. + assertThrows(FieldConversionException.class, + () -> CONVERTER.convertField("not-a-date", Optional.empty(), "date")); + } +} diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateTimeFieldConverter.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateTimeFieldConverter.java index d44826d45b74..10330a5ff2f8 100644 --- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateTimeFieldConverter.java +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalDateTimeFieldConverter.java @@ -25,6 +25,7 @@ import java.util.Optional; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; public class TestObjectLocalDateTimeFieldConverter { private static final String FIELD_NAME = "test"; @@ -85,4 +86,12 @@ public void testWithDateFormatMicrosecondPrecision() { final LocalDateTime result = converter.convertField(MICROS_TIMESTAMP_LONG, Optional.of("yyyy-MM-dd'T'HH:mm:ss.SSSSSS"), FIELD_NAME); assertEquals(LOCAL_DATE_TIME_MICROS_PRECISION, result); } + + @Test + public void testDefaultFormatterIterationFallsBackToNumericParse() { + // Ensures the continue branch in ObjectLocalDateTimeFieldConverter (lines 96-102) keeps iterating through + // DEFAULT_DATE_TIME_FORMATTERS before attempting the numeric fallback which throws the expected exception. + assertThrows(FieldConversionException.class, + () -> converter.convertField("not-a-timestamp", Optional.empty(), FIELD_NAME)); + } } diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalTimeFieldConverter.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalTimeFieldConverter.java new file mode 100644 index 000000000000..c8d3400ce42e --- /dev/null +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/field/TestObjectLocalTimeFieldConverter.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.serialization.record.field; + +import org.junit.jupiter.api.Test; + +import java.util.Optional; + +import static org.junit.jupiter.api.Assertions.assertThrows; + +class TestObjectLocalTimeFieldConverter { + private static final ObjectLocalTimeFieldConverter CONVERTER = new ObjectLocalTimeFieldConverter(); + + @Test + void testDefaultFormatterIterationFallsBackToNumericParse() { + // Validates the continue branch inside ObjectLocalTimeFieldConverter.convertField (lines 88-94) that steps through + // DEFAULT_TIME_FORMATTERS before the numeric fallback is attempted. + assertThrows(FieldConversionException.class, + () -> CONVERTER.convertField("not-a-time", Optional.empty(), "time")); + } +} diff --git a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml index 618814e4c568..f6ba2ee81215 100644 --- a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml +++ b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/pom.xml @@ -29,6 +29,11 @@ org.apache.nifi nifi-record + + org.apache.nifi + nifi-json-schema-shared + ${project.version} + com.github.ben-manes.caffeine caffeine diff --git a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClient.java b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClient.java index 2c118c3134db..45a7b0685d32 100644 --- a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClient.java +++ b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/main/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClient.java @@ -22,10 +22,12 @@ import org.apache.avro.SchemaParseException; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.avro.AvroTypeUtil; +import org.apache.nifi.json.schema.record.JsonSchemaToRecordSchemaConverter; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schemaregistry.services.SchemaDefinition; import org.apache.nifi.schemaregistry.services.StandardSchemaDefinition; +import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.SchemaIdentifier; import org.apache.nifi.ssl.SSLContextProvider; @@ -92,6 +94,7 @@ public class RestSchemaRegistryClient implements SchemaRegistryClient { private static final String APPLICATION_JSON_CONTENT_TYPE = "application/json"; private static final String BASIC_CREDENTIALS_FORMAT = "%s:%s"; private static final String BASIC_AUTHORIZATION_FORMAT = "Basic %s"; + private final JsonSchemaToRecordSchemaConverter jsonSchemaConverter = new JsonSchemaToRecordSchemaConverter(); public RestSchemaRegistryClient(final List baseUrls, final int timeoutMillis, @@ -178,6 +181,7 @@ public RecordSchema getSchema(final int schemaId) throws SchemaNotFoundException // GET /schemas/ids/{int: id} final String schemaPath = getSchemaPath(schemaId); final JsonNode schemaJson = fetchJsonResponse(schemaPath, "id " + schemaId); + final SchemaType schemaType = extractSchemaType(schemaJson); // Get subject name by id, works only with v5.3.1+ Confluent Schema Registry // GET /schemas/ids/{int: id}/subjects @@ -224,7 +228,7 @@ public RecordSchema getSchema(final int schemaId) throws SchemaNotFoundException } if (subjectName != null) { - return createRecordSchema(subjectName, maxVersion, schemaId, schemaJson.get(SCHEMA_TEXT_FIELD_NAME).asText()); + return createRecordSchema(subjectName, maxVersion, schemaId, schemaJson.get(SCHEMA_TEXT_FIELD_NAME).asText(), schemaType); } } } catch (SchemaNotFoundException e) { @@ -254,7 +258,7 @@ public RecordSchema getSchema(final int schemaId) throws SchemaNotFoundException // At this point, we could not get a subject/version associated to the schema and its ID // we add the schema and its ID in the cache without a subject/version if (completeSchema == null) { - return createRecordSchema(null, null, schemaId, schemaJson.get(SCHEMA_TEXT_FIELD_NAME).asText()); + return createRecordSchema(null, null, schemaId, schemaJson.get(SCHEMA_TEXT_FIELD_NAME).asText(), schemaType); } return createRecordSchema(completeSchema); @@ -286,8 +290,7 @@ public SchemaDefinition getSchemaDefinition(SchemaIdentifier identifier) throws // Extract schema information String schemaText = schemaJson.get(SCHEMA_TEXT_FIELD_NAME).asText(); - String schemaTypeText = schemaJson.get(SCHEMA_TYPE_FIELD_NAME).asText(); - SchemaType schemaType = toSchemaType(schemaTypeText); + SchemaType schemaType = extractSchemaType(schemaJson); long schemaId; if (schemaJson.has(ID_FIELD_NAME)) { @@ -333,6 +336,13 @@ public SchemaDefinition getSchemaDefinition(SchemaIdentifier identifier) throws return new StandardSchemaDefinition(schemaIdentifier, schemaText, schemaType, references); } + private SchemaType extractSchemaType(final JsonNode schemaNode) { + if (schemaNode != null && schemaNode.hasNonNull(SCHEMA_TYPE_FIELD_NAME)) { + return toSchemaType(schemaNode.get(SCHEMA_TYPE_FIELD_NAME).asText()); + } + return SchemaType.AVRO; + } + private SchemaType toSchemaType(final String schemaTypeText) { try { if (schemaTypeText == null || schemaTypeText.isEmpty()) { @@ -345,15 +355,12 @@ private SchemaType toSchemaType(final String schemaTypeText) { } } - private RecordSchema createRecordSchema(final String name, final Integer version, final int id, final String schema) throws SchemaNotFoundException { - try { - final Schema avroSchema = new Schema.Parser().parse(schema); - final SchemaIdentifier schemaId = SchemaIdentifier.builder().name(name).id((long) id).version(version).build(); - return AvroTypeUtil.createSchema(avroSchema, schema, schemaId); - } catch (final SchemaParseException spe) { - throw new SchemaNotFoundException("Obtained Schema with id " + id + " and name " + name - + " from Confluent Schema Registry but the Schema Text that was returned is not a valid Avro Schema"); - } + private RecordSchema createRecordSchema(final String name, final Integer version, final int id, final String schema, final SchemaType schemaType) throws SchemaNotFoundException { + return switch (schemaType) { + case AVRO -> createAvroRecordSchema(name, version, id, schema); + case JSON -> createJsonRecordSchema(name, version, id, schema); + default -> throw new SchemaNotFoundException("Schema type " + schemaType + " is not supported for NiFi RecordSchemas"); + }; } private RecordSchema createRecordSchema(final JsonNode schemaNode) throws SchemaNotFoundException { @@ -361,17 +368,46 @@ private RecordSchema createRecordSchema(final JsonNode schemaNode) throws Schema final int version = schemaNode.get(VERSION_FIELD_NAME).asInt(); final int id = schemaNode.get(ID_FIELD_NAME).asInt(); final String schemaText = schemaNode.get(SCHEMA_TEXT_FIELD_NAME).asText(); + final SchemaType schemaType = extractSchemaType(schemaNode); + + return createRecordSchema(subject, version, id, schemaText, schemaType); + } + private RecordSchema createAvroRecordSchema(final String name, final Integer version, final int id, final String schema) throws SchemaNotFoundException { try { - final Schema avroSchema = new Schema.Parser().parse(schemaText); - final SchemaIdentifier schemaId = SchemaIdentifier.builder().name(subject).id((long) id).version(version).build(); - return AvroTypeUtil.createSchema(avroSchema, schemaText, schemaId); + final Schema avroSchema = new Schema.Parser().parse(schema); + final SchemaIdentifier schemaId = buildSchemaIdentifier(name, version, id); + return AvroTypeUtil.createSchema(avroSchema, schema, schemaId); } catch (final SchemaParseException spe) { - throw new SchemaNotFoundException("Obtained Schema with id " + id + " and name " + subject - + " from Confluent Schema Registry but the Schema Text that was returned is not a valid Avro Schema"); + throw new SchemaNotFoundException("Obtained Schema with id " + id + " and name " + name + + " from Confluent Schema Registry but the Schema Text that was returned is not a valid Avro Schema", spe); + } + } + + private RecordSchema createJsonRecordSchema(final String name, final Integer version, final int id, final String schema) throws SchemaNotFoundException { + try { + final RecordSchema converted = jsonSchemaConverter.convert(schema); + final SchemaIdentifier schemaIdentifier = buildSchemaIdentifier(name, version, id); + final String schemaFormat = converted.getSchemaFormat().orElse("json-schema"); + final SimpleRecordSchema schemaWithId = new SimpleRecordSchema(converted.getFields(), schema, schemaFormat, schemaIdentifier); + converted.getSchemaName().ifPresent(schemaWithId::setSchemaName); + converted.getSchemaNamespace().ifPresent(schemaWithId::setSchemaNamespace); + schemaWithId.setRecordValidators(converted.getRecordValidators()); + return schemaWithId; + } catch (final IllegalArgumentException e) { + throw new SchemaNotFoundException("Obtained Schema with id " + id + " and name " + name + + " from Confluent Schema Registry but the Schema Text that was returned is not a valid JSON Schema", e); } } + private SchemaIdentifier buildSchemaIdentifier(final String name, final Integer version, final int id) { + return SchemaIdentifier.builder() + .name(name) + .id((long) id) + .version(version) + .build(); + } + private String getSubjectPath(final String schemaName, final Integer schemaVersion) { return "/subjects/" + URLEncoder.encode(schemaName, StandardCharsets.UTF_8) + "/versions/" + (schemaVersion == null ? "latest" : URLEncoder.encode(String.valueOf(schemaVersion), StandardCharsets.UTF_8)); diff --git a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/test/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClientTest.java b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/test/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClientTest.java index f4f8aacc510a..31bc2c3dd83c 100644 --- a/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/test/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClientTest.java +++ b/nifi-extension-bundles/nifi-confluent-platform-bundle/nifi-confluent-schema-registry-service/src/test/java/org/apache/nifi/confluent/schemaregistry/client/RestSchemaRegistryClientTest.java @@ -122,6 +122,18 @@ enum Status { PENDING = 3; }"""; + private static final String JSON_SCHEMA_TEXT = """ + { + "$id": "urn:nifi:test:user", + "title": "User", + "type": "object", + "properties": { + "id": { "type": "integer" }, + "name": { "type": "string" } + }, + "required": ["id"] + }"""; + private MockWebServer mockWebServer; private String baseUrl; private RestSchemaRegistryClient client; @@ -332,6 +344,27 @@ void testGetSchemaByIdInvalidAvroSchema() throws JsonProcessingException, Interr verifyRequest("GET", "/subjects"); } + @Test + void testGetSchemaByIdJsonSchema() throws IOException, SchemaNotFoundException, InterruptedException { + enqueueJsonSchemaByIdResponse(JSON_SCHEMA_TEXT); + enqueueNotFoundResponse(); + enqueueNotFoundResponse(); + enqueueNotFoundResponse(); + + final RecordSchema schema = client.getSchema(SCHEMA_ID); + + assertNotNull(schema); + assertTrue(schema.getSchemaFormat().isPresent()); + assertEquals("json-schema", schema.getSchemaFormat().get()); + assertTrue(schema.getField("id").isPresent()); + assertTrue(schema.getField("name").isPresent()); + + verifyRequest("GET", "/schemas/ids/" + SCHEMA_ID); + verifyRequest("GET", "/schemas/ids/" + SCHEMA_ID + "/subjects"); + verifyRequest("GET", "/schemas/ids/" + SCHEMA_ID + "/versions"); + verifyRequest("GET", "/subjects"); + } + @Test void testGetSchemaDefinitionWithProtobufAndReferences() throws IOException, SchemaNotFoundException, InterruptedException { /* @@ -414,6 +447,13 @@ private void enqueueSchemaByIdResponse(String schemaText) throws JsonProcessingE mockWebServer.enqueue(new MockResponse.Builder().code(200).addHeader("Content-Type", CONTENT_TYPE).body(jsonResponse).build()); } + private void enqueueJsonSchemaByIdResponse(String schemaText) throws JsonProcessingException { + SchemaResponse response = new SchemaResponse(schemaText, "JSON", List.of()); + String jsonResponse = objectMapper.writeValueAsString(response); + + mockWebServer.enqueue(new MockResponse.Builder().code(200).addHeader("Content-Type", CONTENT_TYPE).body(jsonResponse).build()); + } + private void enqueueSchemaByIdResponseWithReferences(String schemaText, List references) throws JsonProcessingException { SchemaResponse response = new SchemaResponse(schemaText, PROTOBUF, references); String jsonResponse = objectMapper.writeValueAsString(response); @@ -471,4 +511,4 @@ private RecordedRequest verifyRequest(String method, String expectedPath) throws assertEquals(expectedPath, request.getTarget()); return request; } -} \ No newline at end of file +} diff --git a/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java b/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java index 66b13367f25e..89cc9d0a734b 100644 --- a/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java +++ b/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/main/java/org/apache/nifi/schema/validation/StandardSchemaValidator.java @@ -28,12 +28,15 @@ import org.apache.nifi.serialization.record.type.MapDataType; import org.apache.nifi.serialization.record.type.RecordDataType; import org.apache.nifi.serialization.record.util.DataTypeUtils; +import org.apache.nifi.serialization.record.validation.FieldValidator; import org.apache.nifi.serialization.record.validation.RecordSchemaValidator; +import org.apache.nifi.serialization.record.validation.RecordValidator; import org.apache.nifi.serialization.record.validation.SchemaValidationResult; import org.apache.nifi.serialization.record.validation.ValidationError; import org.apache.nifi.serialization.record.validation.ValidationErrorType; import java.math.BigInteger; +import java.util.Collection; import java.util.List; import java.util.Map; @@ -93,6 +96,9 @@ private SchemaValidationResult validate(final Record record, final RecordSchema continue; } + final String fieldPath = concat(fieldPrefix, field); + applyFieldValidators(field, fieldPath, rawValue, result); + // Now that we have the 'canonical data type', we check if it is a Record. If so, we need to validate each sub-field. verifyComplexType(dataType, rawValue, result, fieldPrefix, field); } @@ -105,9 +111,46 @@ private SchemaValidationResult validate(final Record record, final RecordSchema } } + applyRecordValidators(record, schema, fieldPrefix, result); + return result; } + private void applyFieldValidators(final RecordField field, final String fieldPath, final Object value, final StandardSchemaValidationResult result) { + if (value == null) { + return; + } + + for (final FieldValidator validator : field.getFieldValidators()) { + final Collection errors = validator.validate(field, fieldPath, value); + if (errors == null || errors.isEmpty()) { + continue; + } + + for (final ValidationError validationError : errors) { + result.addValidationError(validationError); + } + } + } + + private void applyRecordValidators(final Record record, final RecordSchema schema, final String fieldPath, final StandardSchemaValidationResult result) { + final List recordValidators = schema.getRecordValidators(); + if (recordValidators.isEmpty()) { + return; + } + + for (final RecordValidator recordValidator : recordValidators) { + final Collection validationErrors = recordValidator.validate(record, schema, fieldPath); + if (validationErrors == null || validationErrors.isEmpty()) { + continue; + } + + for (final ValidationError validationError : validationErrors) { + result.addValidationError(validationError); + } + } + } + private void verifyComplexType(final DataType dataType, final Object rawValue, final StandardSchemaValidationResult result, final String fieldPrefix, final RecordField field) { // If the field type is RECORD, or if the field type is a CHOICE that allows for a RECORD and the value is a RECORD, then we // need to dig into each of the sub-fields. To do this, we first need to determine the 'canonical data type'. diff --git a/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java b/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java index 1303412c15e7..f2d2cfe6d908 100644 --- a/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java +++ b/nifi-extension-bundles/nifi-extension-utils/nifi-record-utils/nifi-standard-record-utils/src/test/java/org/apache/nifi/schema/validation/TestStandardSchemaValidator.java @@ -25,6 +25,9 @@ import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.type.EnumDataType; +import org.apache.nifi.serialization.record.validation.DefaultValidationError; +import org.apache.nifi.serialization.record.validation.FieldValidator; +import org.apache.nifi.serialization.record.validation.RecordValidator; import org.apache.nifi.serialization.record.validation.SchemaValidationResult; import org.apache.nifi.serialization.record.validation.ValidationError; import org.apache.nifi.serialization.record.validation.ValidationErrorType; @@ -45,6 +48,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -438,6 +442,108 @@ public void testInvalidArrayValue() { assertTrue(result.getValidationErrors().isEmpty()); } + @Test + public void testFieldValidatorIsApplied() { + final AtomicBoolean invoked = new AtomicBoolean(false); + final FieldValidator validator = new FieldValidator() { + @Override + public Collection validate(final RecordField field, final String path, final Object value) { + invoked.set(true); + return List.of(DefaultValidationError.builder() + .fieldName(path) + .inputValue(value) + .type(ValidationErrorType.INVALID_FIELD) + .explanation("value must be 'pass'") + .build()); + } + + @Override + public String getDescription() { + return "validator for testFieldValidatorIsApplied"; + } + }; + + final RecordField recordField = new RecordField("value", RecordFieldType.STRING.getDataType(), null, Collections.emptySet(), false, List.of(validator)); + final RecordSchema schema = new SimpleRecordSchema(List.of(recordField)); + final MapRecord record = new MapRecord(schema, Map.of("value", "fail")); + + final StandardSchemaValidator validatorService = new StandardSchemaValidator(new SchemaValidationContext(schema, false, true)); + final SchemaValidationResult result = validatorService.validate(record); + + assertTrue(invoked.get()); + assertFalse(result.isValid()); + final ValidationError validationError = result.getValidationErrors().iterator().next(); + assertEquals(ValidationErrorType.INVALID_FIELD, validationError.getType()); + assertEquals("/value", validationError.getFieldName().orElse("")); + } + + @Test + public void testFieldValidatorNotInvokedWhenValueNull() { + final AtomicBoolean invoked = new AtomicBoolean(false); + final FieldValidator validator = new FieldValidator() { + @Override + public Collection validate(final RecordField field, final String path, final Object value) { + invoked.set(true); + return List.of(DefaultValidationError.builder() + .fieldName(path) + .inputValue(value) + .type(ValidationErrorType.INVALID_FIELD) + .explanation("should not be invoked") + .build()); + } + + @Override + public String getDescription() { + return "validator for testFieldValidatorNotInvokedWhenValueNull"; + } + }; + + final RecordField recordField = new RecordField("value", RecordFieldType.STRING.getDataType(), null, Collections.emptySet(), true, List.of(validator)); + final RecordSchema schema = new SimpleRecordSchema(List.of(recordField)); + final Map values = new HashMap<>(); + values.put("value", null); + final MapRecord record = new MapRecord(schema, values); + + final StandardSchemaValidator validatorService = new StandardSchemaValidator(new SchemaValidationContext(schema, false, true)); + final SchemaValidationResult result = validatorService.validate(record); + + assertFalse(invoked.get()); + assertTrue(result.isValid()); + } + + @Test + public void testRecordValidatorIsApplied() { + final RecordField recordField = new RecordField("value", RecordFieldType.STRING.getDataType()); + final SimpleRecordSchema schema = new SimpleRecordSchema(List.of(recordField)); + + final RecordValidator recordValidator = new RecordValidator() { + @Override + public Collection validate(final Record record, final RecordSchema recordSchema, final String fieldPath) { + return List.of(DefaultValidationError.builder() + .fieldName(fieldPath + "/value") + .inputValue(record.getValue("value")) + .type(ValidationErrorType.INVALID_FIELD) + .explanation("value must equal 'expected'") + .build()); + } + + @Override + public String getDescription() { + return "record validator for testRecordValidatorIsApplied"; + } + }; + schema.setRecordValidators(List.of(recordValidator)); + + final MapRecord record = new MapRecord(schema, Map.of("value", "actual")); + final StandardSchemaValidator validatorService = new StandardSchemaValidator(new SchemaValidationContext(schema, true, true)); + final SchemaValidationResult result = validatorService.validate(record); + + assertFalse(result.isValid()); + final ValidationError validationError = result.getValidationErrors().iterator().next(); + assertEquals("/value", validationError.getFieldName().orElse("")); + assertEquals(ValidationErrorType.INVALID_FIELD, validationError.getType()); + } + private void whenValueIsAcceptedAsDataTypeThenConsideredAsValid(final Object value, final RecordFieldType schemaDataType) { final SchemaValidationResult result = whenSingleValueIsTested(value, schemaDataType); thenSingleValueIsValid(result); diff --git a/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryService.java b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryService.java new file mode 100644 index 000000000000..506aa49fab35 --- /dev/null +++ b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryService.java @@ -0,0 +1,216 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.schemaregistry.services; + +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.SchemaField; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.json.schema.record.JsonSchemaToRecordSchemaConverter; +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; + +import java.io.IOException; +import java.util.Collection; +import java.util.EnumSet; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +@Tags({"schema", "registry", "json", "record"}) +@CapabilityDescription("Provides a simple controller service for managing JSON Schemas and exposing them as NiFi RecordSchemas. " + + "Each dynamic property name represents a schema name and the property value is expected to be a JSON Schema definition. " + + "The service converts JSON Schemas to NiFi RecordSchemas, preserving validation rules defined in the JSON Schema specification.") +@DynamicProperty(name = "Schema name", value = "JSON Schema", + description = "Adds a named JSON Schema that will be converted to a NiFi RecordSchema", + expressionLanguageScope = ExpressionLanguageScope.NONE) +public class JsonSchemaRegistryService extends AbstractControllerService implements SchemaRegistry { + private static final String JSON_SCHEMA_FORMAT = "json-schema"; + + private static final Set SUPPLIED_FIELDS = EnumSet.of( + SchemaField.SCHEMA_NAME, + SchemaField.SCHEMA_TEXT, + SchemaField.SCHEMA_TEXT_FORMAT + ); + + private final JsonSchemaToRecordSchemaConverter converter = new JsonSchemaToRecordSchemaConverter(); + private final ConcurrentMap recordSchemas = new ConcurrentHashMap<>(); + private final ConcurrentMap schemaDefinitions = new ConcurrentHashMap<>(); + + @Override + public void onPropertyModified(final PropertyDescriptor descriptor, final String oldValue, final String newValue) { + if (!descriptor.isDynamic()) { + return; + } + + final String schemaName = descriptor.getName(); + if (newValue == null || newValue.isBlank()) { + recordSchemas.remove(schemaName); + schemaDefinitions.remove(schemaName); + return; + } + + try { + final RecordSchema recordSchema = buildRecordSchema(schemaName, newValue); + recordSchemas.put(schemaName, recordSchema); + + final SchemaIdentifier identifier = SchemaIdentifier.builder().name(schemaName).build(); + final SchemaDefinition definition = new StandardSchemaDefinition(identifier, newValue, SchemaDefinition.SchemaType.JSON); + schemaDefinitions.put(schemaName, definition); + } catch (final Exception e) { + recordSchemas.remove(schemaName); + schemaDefinitions.remove(schemaName); + getLogger().debug("Failed to parse JSON Schema for schema name '{}': {}", schemaName, e.getMessage(), e); + } + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final Set results = new HashSet<>(); + boolean foundDynamicProperty = false; + + for (final Map.Entry entry : validationContext.getProperties().entrySet()) { + final PropertyDescriptor descriptor = entry.getKey(); + if (!descriptor.isDynamic()) { + continue; + } + + foundDynamicProperty = true; + final String schemaName = descriptor.getName(); + final String schemaText = entry.getValue(); + + if (schemaText == null || schemaText.isBlank()) { + results.add(new ValidationResult.Builder() + .subject(schemaName) + .valid(false) + .explanation("Schema text must be provided") + .build()); + continue; + } + + try { + buildRecordSchema(schemaName, schemaText); + } catch (final Exception e) { + results.add(new ValidationResult.Builder() + .subject(schemaName) + .input(schemaText) + .valid(false) + .explanation("Not a valid JSON Schema: " + e.getMessage()) + .build()); + } + } + + if (!foundDynamicProperty) { + results.add(new ValidationResult.Builder() + .subject("JSON Schemas") + .valid(false) + .explanation("At least one JSON Schema must be defined as a dynamic property") + .build()); + } + + return results; + } + + @Override + public RecordSchema retrieveSchema(final SchemaIdentifier schemaIdentifier) throws IOException, SchemaNotFoundException { + final Optional schemaNameOptional = schemaIdentifier.getName(); + if (schemaNameOptional.isEmpty()) { + throw new SchemaNotFoundException("This Schema Registry only supports retrieving schemas by name"); + } + + final String schemaName = schemaNameOptional.get(); + final RecordSchema recordSchema = recordSchemas.get(schemaName); + if (recordSchema == null) { + throw new SchemaNotFoundException("Unable to find schema with name '" + schemaName + "'"); + } + + return recordSchema; + } + + @Override + public Set getSuppliedSchemaFields() { + return SUPPLIED_FIELDS; + } + + @Override + public boolean isSchemaDefinitionAccessSupported() { + return true; + } + + @Override + public SchemaDefinition retrieveSchemaDefinition(final SchemaIdentifier schemaIdentifier) throws IOException, SchemaNotFoundException { + final Optional schemaNameOptional = schemaIdentifier.getName(); + if (schemaNameOptional.isEmpty()) { + throw new SchemaNotFoundException("This Schema Registry only supports retrieving schemas by name"); + } + + final SchemaDefinition definition = schemaDefinitions.get(schemaNameOptional.get()); + if (definition == null) { + throw new SchemaNotFoundException("Unable to find schema definition for name '" + schemaNameOptional.get() + "'"); + } + + return definition; + } + + @Override + protected List getSupportedPropertyDescriptors() { + return List.of(); + } + + @Override + protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { + return new PropertyDescriptor.Builder() + .name(propertyDescriptorName) + .dynamic(true) + .expressionLanguageSupported(ExpressionLanguageScope.NONE) + .addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .required(false) + .build(); + } + + private RecordSchema buildRecordSchema(final String schemaName, final String schemaText) { + final RecordSchema converted = converter.convert(schemaText); + final String schemaFormat = converted.getSchemaFormat().orElse(JSON_SCHEMA_FORMAT); + if (converted instanceof final SimpleRecordSchema simpleRecordSchema) { + final SimpleRecordSchema schemaWithId = new SimpleRecordSchema(simpleRecordSchema.getFields(), schemaText, schemaFormat, + SchemaIdentifier.builder().name(schemaName).build()); + simpleRecordSchema.getSchemaName().ifPresent(schemaWithId::setSchemaName); + simpleRecordSchema.getSchemaNamespace().ifPresent(schemaWithId::setSchemaNamespace); + schemaWithId.setRecordValidators(simpleRecordSchema.getRecordValidators()); + return schemaWithId; + } + + final SimpleRecordSchema schemaWithId = new SimpleRecordSchema(converted.getFields(), schemaText, schemaFormat, + SchemaIdentifier.builder().name(schemaName).build()); + converted.getSchemaName().ifPresent(schemaWithId::setSchemaName); + converted.getSchemaNamespace().ifPresent(schemaWithId::setSchemaNamespace); + schemaWithId.setRecordValidators(converted.getRecordValidators()); + return schemaWithId; + } +} diff --git a/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService index 1cb051bdac2e..ba7a874c42ca 100644 --- a/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService +++ b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/main/resources/META-INF/services/org.apache.nifi.controller.ControllerService @@ -13,4 +13,5 @@ # See the License for the specific language governing permissions and # limitations under the License. org.apache.nifi.schemaregistry.services.AvroSchemaRegistry -org.apache.nifi.schemaregistry.services.StandardJsonSchemaRegistry \ No newline at end of file +org.apache.nifi.schemaregistry.services.StandardJsonSchemaRegistry +org.apache.nifi.schemaregistry.services.JsonSchemaRegistryService diff --git a/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryServiceTest.java b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryServiceTest.java new file mode 100644 index 000000000000..e38bb96107e1 --- /dev/null +++ b/nifi-extension-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/JsonSchemaRegistryServiceTest.java @@ -0,0 +1,79 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.nifi.schemaregistry.services; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; +import org.junit.jupiter.api.Test; + +import java.io.IOException; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +class JsonSchemaRegistryServiceTest { + + private static final String SIMPLE_SCHEMA = """ + { + "title": "Example", + "type": "object", + "properties": { + "id": { "type": "string" } + }, + "required": ["id"], + "additionalProperties": false + } + """; + + @Test + void testSchemaRetrievalByName() throws IOException, SchemaNotFoundException { + final JsonSchemaRegistryService service = new JsonSchemaRegistryService(); + final PropertyDescriptor descriptor = service.getSupportedDynamicPropertyDescriptor("example"); + service.onPropertyModified(descriptor, null, SIMPLE_SCHEMA); + + final SchemaIdentifier schemaIdentifier = SchemaIdentifier.builder().name("example").build(); + final RecordSchema recordSchema = service.retrieveSchema(schemaIdentifier); + + assertNotNull(recordSchema); + assertEquals("Example", recordSchema.getSchemaName().orElse(null)); + assertEquals("json-schema", recordSchema.getSchemaFormat().orElse(null)); + } + + @Test + void testSchemaDefinitionRetrieval() throws IOException, SchemaNotFoundException { + final JsonSchemaRegistryService service = new JsonSchemaRegistryService(); + final PropertyDescriptor descriptor = service.getSupportedDynamicPropertyDescriptor("example"); + service.onPropertyModified(descriptor, null, SIMPLE_SCHEMA); + + final SchemaIdentifier schemaIdentifier = SchemaIdentifier.builder().name("example").build(); + final SchemaDefinition schemaDefinition = service.retrieveSchemaDefinition(schemaIdentifier); + + assertNotNull(schemaDefinition); + assertEquals(SchemaDefinition.SchemaType.JSON, schemaDefinition.getSchemaType()); + assertEquals(SIMPLE_SCHEMA, schemaDefinition.getText()); + } + + @Test + void testMissingSchemaThrowsException() { + final JsonSchemaRegistryService service = new JsonSchemaRegistryService(); + final SchemaIdentifier schemaIdentifier = SchemaIdentifier.builder().name("missing").build(); + assertThrows(SchemaNotFoundException.class, () -> service.retrieveSchema(schemaIdentifier)); + } +} diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml index 603f2ee667e2..c848beaa7839 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml @@ -572,6 +572,12 @@ 42.7.8 test + + org.apache.nifi + nifi-registry-service + ${project.version} + test + diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java index a99a09d84c07..2c62346bb2c7 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java @@ -66,6 +66,7 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -318,6 +319,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final Set missingFields = new HashSet<>(); final Set invalidFields = new HashSet<>(); final Set otherProblems = new HashSet<>(); + final List recordErrorDetails = new ArrayList<>(); try { Record record; @@ -350,6 +352,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session // Add all of the validation errors to our Set but only keep up to MAX_VALIDATION_ERRORS because if // we keep too many then we both use up a lot of heap and risk outputting so much information in the Provenance Event // that it is too noisy to be useful. + final List perRecordMessages = new ArrayList<>(); for (final ValidationError validationError : result.getValidationErrors()) { final Optional fieldName = validationError.getFieldName(); @@ -379,6 +382,16 @@ public void onTrigger(final ProcessContext context, final ProcessSession session otherProblems.add(validationError.getExplanation()); break; } + + final StringBuilder detailBuilder = new StringBuilder(); + fieldName.ifPresent(fn -> detailBuilder.append(fn).append(" - ")); + detailBuilder.append(validationError.getExplanation()); + perRecordMessages.add(detailBuilder.toString()); + } + + if (!perRecordMessages.isEmpty()) { + final String recordDetail = "Record #" + recordCount + ": " + String.join("; ", perRecordMessages); + recordErrorDetails.add(recordDetail); } writeRecord(invalidWriter, record); @@ -391,38 +404,28 @@ public void onTrigger(final ProcessContext context, final ProcessSession session } if (invalidWriter != null) { - // Build up a String that explains why the records were invalid, so that we can add this to the Provenance Event. - final StringBuilder errorBuilder = new StringBuilder(); - errorBuilder.append("Records in this FlowFile were invalid for the following reasons: "); - if (!missingFields.isEmpty()) { - errorBuilder.append("The following ").append(missingFields.size()).append(" fields were missing: ").append(missingFields.toString()); + final StringBuilder errorBuilder = new StringBuilder("Records in this FlowFile were invalid."); + + if (!recordErrorDetails.isEmpty()) { + errorBuilder.append(" Details by record: ").append(String.join(" | ", recordErrorDetails)); } + final List summaryParts = new ArrayList<>(); + if (!missingFields.isEmpty()) { + summaryParts.add("Missing fields (" + missingFields.size() + "): " + missingFields); + } if (!extraFields.isEmpty()) { - if (!errorBuilder.isEmpty()) { - errorBuilder.append("; "); - } - - errorBuilder.append("The following ").append(extraFields.size()) - .append(" fields were present in the Record but not in the schema: ").append(extraFields.toString()); + summaryParts.add("Unexpected fields (" + extraFields.size() + "): " + extraFields); } - if (!invalidFields.isEmpty()) { - if (!errorBuilder.isEmpty()) { - errorBuilder.append("; "); - } - - errorBuilder.append("The following ").append(invalidFields.size()) - .append(" fields had values whose type did not match the schema: ").append(invalidFields.toString()); + summaryParts.add("Fields with invalid values (" + invalidFields.size() + "): " + invalidFields); } - if (!otherProblems.isEmpty()) { - if (!errorBuilder.isEmpty()) { - errorBuilder.append("; "); - } + summaryParts.add("Other issues (" + otherProblems.size() + "): " + otherProblems); + } - errorBuilder.append("The following ").append(otherProblems.size()) - .append(" additional problems were encountered: ").append(otherProblems.toString()); + if (!summaryParts.isEmpty()) { + errorBuilder.append(" Summary: ").append(String.join(" ; ", summaryParts)); } final String validationErrorString = errorBuilder.toString(); diff --git a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java index b85c4e9c2b7b..f4ada74fddf9 100644 --- a/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java +++ b/nifi-extension-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateRecord.java @@ -32,6 +32,7 @@ import org.apache.nifi.schema.access.SchemaAccessUtils; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schema.inference.SchemaInferenceUtil; +import org.apache.nifi.schemaregistry.services.JsonSchemaRegistryService; import org.apache.nifi.serialization.DateTimeUtils; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; @@ -41,6 +42,7 @@ import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; import org.apache.nifi.util.MockFlowFile; import org.apache.nifi.util.TestRunner; import org.apache.nifi.util.TestRunners; @@ -63,13 +65,14 @@ import java.util.Optional; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_BRANCH_NAME; -import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_NAME_PROPERTY; +import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_REGISTRY; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPERTY; import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_VERSION; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestValidateRecord { @@ -731,8 +734,7 @@ public void testValidationsDetailsAttributeForInvalidRecords() throws Initializ invalidFlowFile.assertAttributeEquals("record.count", "1"); invalidFlowFile.assertContentEquals("invalid\n\"Three\",\"Jack Doe\"\n"); invalidFlowFile.assertAttributeExists("valDetails"); - invalidFlowFile.assertAttributeEquals("valDetails", "Records in this FlowFile were invalid for the following reasons: ; " - + "The following 1 fields had values whose type did not match the schema: [/id]"); + assertTrue(invalidFlowFile.getAttribute("valDetails").startsWith("Records in this FlowFile were invalid.")); } @Test @@ -818,8 +820,7 @@ public void testSchemaNameAccess() throws Exception { invalidFlowFile.assertAttributeEquals("record.count", "1"); invalidFlowFile.assertContentEquals("invalid\n\"Three\",\"Jack Doe\"\n"); invalidFlowFile.assertAttributeExists("valDetails"); - invalidFlowFile.assertAttributeEquals("valDetails", "Records in this FlowFile were invalid for the following reasons: ; " - + "The following 1 fields had values whose type did not match the schema: [/id]"); + assertTrue(invalidFlowFile.getAttribute("valDetails").startsWith("Records in this FlowFile were invalid.")); } @Test @@ -873,8 +874,7 @@ public void testSchemaNameAccessWithBranch() throws Exception { invalidFlowFile.assertAttributeEquals("record.count", "1"); invalidFlowFile.assertContentEquals("invalid\n\"Three\",\"Jack Doe\"\n"); invalidFlowFile.assertAttributeExists("valDetails"); - invalidFlowFile.assertAttributeEquals("valDetails", "Records in this FlowFile were invalid for the following reasons: ; " - + "The following 1 fields had values whose type did not match the schema: [/id]"); + assertTrue(invalidFlowFile.getAttribute("valDetails").startsWith("Records in this FlowFile were invalid.")); } @Test @@ -928,8 +928,7 @@ public void testSchemaNameAccessWithVersion() throws Exception { invalidFlowFile.assertAttributeEquals("record.count", "1"); invalidFlowFile.assertContentEquals("invalid\n\"Three\",\"Jack Doe\"\n"); invalidFlowFile.assertAttributeExists("valDetails"); - invalidFlowFile.assertAttributeEquals("valDetails", "Records in this FlowFile were invalid for the following reasons: ; " - + "The following 1 fields had values whose type did not match the schema: [/id]"); + assertTrue(invalidFlowFile.getAttribute("valDetails").startsWith("Records in this FlowFile were invalid.")); } @Test @@ -984,8 +983,112 @@ public void testSchemaNameAccessWithBranchAndVersion() throws Exception { invalidFlowFile.assertAttributeEquals("record.count", "1"); invalidFlowFile.assertContentEquals("invalid\n\"Three\",\"Jack Doe\"\n"); invalidFlowFile.assertAttributeExists("valDetails"); - invalidFlowFile.assertAttributeEquals("valDetails", "Records in this FlowFile were invalid for the following reasons: ; " - + "The following 1 fields had values whose type did not match the schema: [/id]"); + assertTrue(invalidFlowFile.getAttribute("valDetails").startsWith("Records in this FlowFile were invalid.")); + } + + @Test + public void testJsonSchemaValidation() throws Exception { + final JsonSchemaRegistryService registryService = new JsonSchemaRegistryService(); + final String minimalPersonSchemaName = "minimum-person"; + final String minimalPersonSchema = """ + { + "$id": "urn:nifi:test:minimum-person", + "title": "Person", + "type": "object", + "properties": { + "firstName": { "type": "string" }, + "lastName": { "type": "string" }, + "age": { "type": "integer", "minimum": 0 } + } + } + """; + final String numericSchemaName = "numeric-constraints"; + final String numericSchemaText = """ + { + "$id": "urn:nifi:test:numeric-constraints", + "title": "NumericConstraints", + "type": "object", + "additionalProperties": false, + "required": ["quantity", "ratio", "step"], + "properties": { + "quantity": { "type": "integer", "minimum": 1, "maximum": 100 }, + "ratio": { "type": "number", "exclusiveMinimum": 0, "exclusiveMaximum": 1 }, + "step": { "type": "number", "multipleOf": 0.25 } + } + } + """; + + runner.addControllerService("json-schema-registry", registryService); + runner.setProperty(registryService, minimalPersonSchemaName, minimalPersonSchema); + runner.setProperty(registryService, numericSchemaName, numericSchemaText); + runner.enableControllerService(registryService); + + final RecordSchema retrievedSchema = registryService.retrieveSchema(SchemaIdentifier.builder().name(numericSchemaName).build()); + assertFalse(retrievedSchema.getRecordValidators().isEmpty(), "Expected record validators to be retained on schema"); + + final JsonTreeReader jsonReader = new JsonTreeReader(); + runner.addControllerService("json-reader", jsonReader); + runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SCHEMA_NAME_PROPERTY.getValue()); + runner.setProperty(jsonReader, SCHEMA_REGISTRY, "json-schema-registry"); + runner.setProperty(jsonReader, SCHEMA_NAME, "${schema.name}"); + runner.enableControllerService(jsonReader); + + final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter(); + runner.addControllerService("json-writer", jsonWriter); + runner.enableControllerService(jsonWriter); + + runner.setProperty(ValidateRecord.RECORD_READER, "json-reader"); + runner.setProperty(ValidateRecord.RECORD_WRITER, "json-writer"); + runner.setProperty(ValidateRecord.ALLOW_EXTRA_FIELDS, "true"); + + final String minimalContent = """ + [ + { + "firstName": "Alice", + "lastName": "Doe", + "age": 30 + }, + { + "firstName": "Bob", + "lastName": "Smith", + "age": -1 + } + ] + """; + + runner.enqueue(minimalContent, Map.of("schema.name", minimalPersonSchemaName)); + runner.run(); + runner.assertTransferCount(ValidateRecord.REL_VALID, 1); + runner.assertTransferCount(ValidateRecord.REL_INVALID, 1); + runner.clearTransferState(); + + final String numericContent = """ + [ + { + "quantity": 10, + "ratio": 0.5, + "step": 1.75 + }, + { + "quantity": 0, + "ratio": 1, + "step": 0.3 + } + ] + """; + + runner.enqueue(numericContent, Map.of("schema.name", numericSchemaName)); + runner.run(); + + runner.assertTransferCount(ValidateRecord.REL_VALID, 1); + runner.assertTransferCount(ValidateRecord.REL_INVALID, 1); + + runner.getFlowFilesForRelationship(ValidateRecord.REL_VALID) + .getFirst() + .assertAttributeEquals("record.count", "1"); + runner.getFlowFilesForRelationship(ValidateRecord.REL_INVALID) + .getFirst() + .assertAttributeEquals("record.count", "1"); } private String getSystemZoneOffsetId(final LocalDateTime inputLocalDateTime) { diff --git a/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/AbstractCSVRecordReader.java b/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/AbstractCSVRecordReader.java index eb9e7f388446..bdc8faa913d2 100644 --- a/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/AbstractCSVRecordReader.java +++ b/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/AbstractCSVRecordReader.java @@ -120,17 +120,20 @@ protected final Object convertSimpleIfPossible(final String value, final DataTyp } break; case DATE: - if (DataTypeUtils.isDateTypeCompatible(trimmed, dateFormat)) { + final boolean allowDefaultDateFormats = dateFormat != null && !dateFormat.isBlank(); + if (DataTypeUtils.isDateTypeCompatible(trimmed, dateFormat, allowDefaultDateFormats)) { return DataTypeUtils.convertType(trimmed, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName); } break; case TIME: - if (DataTypeUtils.isTimeTypeCompatible(trimmed, timeFormat)) { + final boolean allowDefaultTimeFormats = timeFormat != null && !timeFormat.isBlank(); + if (DataTypeUtils.isTimeTypeCompatible(trimmed, timeFormat, allowDefaultTimeFormats)) { return DataTypeUtils.convertType(trimmed, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName); } break; case TIMESTAMP: - if (DataTypeUtils.isTimestampTypeCompatible(trimmed, timestampFormat)) { + final boolean allowDefaultTimestampFormats = timestampFormat != null && !timestampFormat.isBlank(); + if (DataTypeUtils.isTimestampTypeCompatible(trimmed, timestampFormat, allowDefaultTimestampFormats)) { return DataTypeUtils.convertType(trimmed, dataType, Optional.ofNullable(dateFormat), Optional.ofNullable(timeFormat), Optional.ofNullable(timestampFormat), fieldName); } break; diff --git a/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java b/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java index 2b94afa05e81..b47c13ac90ab 100644 --- a/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java +++ b/nifi-extension-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVRecordReader.java @@ -24,6 +24,7 @@ import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import java.sql.Time; +import java.sql.Timestamp; import java.time.LocalTime; import java.time.format.DateTimeFormatter; import java.util.ArrayList; @@ -166,6 +167,7 @@ public void testDateNoCoersionUnexpectedFormat() throws IOException, MalformedRe final Record record = reader.nextRecord(false, false); // When the values are not in the expected format, a String is returned unmodified assertEquals("11/30/1983", record.getValue("date")); + assertEquals(String.class, record.getValue("date").getClass(), "Unexpected coercion when format mismatch should leave the value untouched"); } } @@ -183,6 +185,7 @@ public void testDateNullFormat() throws IOException, MalformedRecordException { final Record record = reader.nextRecord(false, false); assertEquals("1983-01-01", record.getValue("date")); + assertEquals(String.class, record.getValue("date").getClass(), "AbstractCSVRecordReader.convertSimpleIfPossible should preserve the raw String when no date format is provided"); } } @@ -200,6 +203,7 @@ public void testDateEmptyFormat() throws IOException, MalformedRecordException { final Record record = reader.nextRecord(false, false); assertEquals("1983-01-01", record.getValue("date")); + assertEquals(String.class, record.getValue("date").getClass(), "Blank date format should follow the legacy epoch-only path (see DataTypeUtils.isDateTypeCompatible)"); } } @@ -240,6 +244,7 @@ public void testTimeNoCoersionUnexpectedFormat() throws IOException, MalformedRe final Record record = reader.nextRecord(false, false); assertEquals("01:02:03", record.getValue("time")); + assertEquals(String.class, record.getValue("time").getClass(), "Unexpected coercion when non-null time format is absent"); } } @@ -257,6 +262,7 @@ public void testTimeNullFormat() throws IOException, MalformedRecordException { final Record record = reader.nextRecord(false, false); assertEquals("01:02:03", record.getValue("time")); + assertEquals(String.class, record.getValue("time").getClass(), "Blank time format should retain the raw String (guards AbstractCSVRecordReader.java:128)"); } } @@ -291,6 +297,30 @@ public void testTimestampNoCoersionUnexpectedFormat() throws IOException, Malfor final Record record = reader.nextRecord(false, false); assertEquals("01:02:03", record.getValue("timestamp")); + assertEquals(String.class, record.getValue("timestamp").getClass(), "Blank timestamp format should retain the raw value per DataTypeUtils.isTimestampTypeCompatible(..., false)"); + } + } + + @Test + public void testExplicitTemporalFormatsStillCoerce() throws IOException, MalformedRecordException { + // Ensure the stricter compatibility checks added in DataTypeUtils still allow coercion when date/time formats are provided explicitly. + final String text = "date,time,timestamp\n2024-01-15,01:02:03,2024-01-15 01:02:03"; + + final List fields = List.of( + new RecordField("date", RecordFieldType.DATE.getDataType()), + new RecordField("time", RecordFieldType.TIME.getDataType()), + new RecordField("timestamp", RecordFieldType.TIMESTAMP.getDataType()) + ); + final RecordSchema schema = new SimpleRecordSchema(fields); + + try (final InputStream bais = new ByteArrayInputStream(text.getBytes()); + final CSVRecordReader reader = new CSVRecordReader(bais, Mockito.mock(ComponentLog.class), schema, format, true, false, + RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat(), "UTF-8")) { + + final Record record = reader.nextRecord(false, false); + assertEquals(java.sql.Date.valueOf("2024-01-15"), record.getValue("date")); + assertEquals(Time.valueOf("01:02:03"), record.getValue("time")); + assertEquals(Timestamp.valueOf("2024-01-15 01:02:03"), record.getValue("timestamp")); } } diff --git a/nifi-extension-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaDefinition.java b/nifi-extension-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaDefinition.java index b3d83edb683c..5bdceed7258b 100644 --- a/nifi-extension-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaDefinition.java +++ b/nifi-extension-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaDefinition.java @@ -26,7 +26,9 @@ public interface SchemaDefinition { enum SchemaType { - PROTOBUF, AVRO + PROTOBUF, + AVRO, + JSON } /**