You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/09/28 16:23:28 UTC

[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6337: NIFI-7392: Add ValidateJson processor to standard bundle

exceptionfactory commented on code in PR #6337:
URL: https://github.com/apache/nifi/pull/6337#discussion_r982599630


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-nar/src/main/resources/META-INF/NOTICE:
##########
@@ -248,6 +248,10 @@ The following binary components are provided under the Apache Software License v
       Apache MINA Core 2.0.16
       Copyright 2004-2016 Apache MINA Project
 
+  (ASLv2) Json-schema-validator 
+   The following NOTICE information applies:
+      Copyright (c) 2019 Network New Technologies Inc.
+

Review Comment:
   This addition to the NOTICE file should not be necessary since JSON Schema Validator follows the standard Apache License 2.0 and does not include any specific notices except for third party libraries, which are already covered.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }

Review Comment:
   It would be helpful to refactor this approach to use a separate Java `enum`, which would support parsing the property instead of including these conditionals.  The `enum` should implement `DescribedValue`, which will allow using it for the `allowableValues` in the Property Descriptor. The `enum` can also have a `getVersionFlag()` method to return the JSON Schema VersionFlag value.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed

Review Comment:
   This comment is not really necessary as the status is indicated through log messages. Recommend removing this comment and other comments in other blocks.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";

Review Comment:
   Recommend renaming this property to remove the `validatejson` prefix. What do you think about `json.validation.errors`?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);

Review Comment:
   This approach requires loading and parsing the Schema on each invocation, which is not very efficient. Recommend parsing and storing the `JsonSchema` instance in `onScheduled()`, which will make Processor execution much faster.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);

Review Comment:
   This is not necessary because of the call to `session.transfer()`.
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);
+                session.transfer(flowFile, REL_INVALID);
+            } else {
+                // Schema check passed
+                this.getLogger().debug("Successfully validated {} against schema; routing to 'valid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_VALID);
+                session.transfer(flowFile, REL_VALID);
+            }
+
+        } catch (IOException ioe) {
+            // Failed to read flowFile
+            this.getLogger().error("Failed to process {}, routing to 'failure' details:", flowFile, ioe);

Review Comment:
   The trailing `:` character should be removed, and the message should be adjusted:
   ```suggestion
               this.getLogger().error("JSON processing failed {}", flowFile, ioe);
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")

Review Comment:
   Property names in general should either be all lowercase, or should follow the same conventions as the Display Name. For this reason, recommend changing to `Schema Version`.
   ```suggestion
           .Builder().name("Schema Version")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)

Review Comment:
   The Property Descriptor includes a newer feature called Resource References. This will support reading the JSON Schema from either a File or from a direct content. The value can be retrieved using `asResourceReference()` in `onTrigger`. This would make the configuration more flexible.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);
+                session.transfer(flowFile, REL_INVALID);
+            } else {
+                // Schema check passed

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")

Review Comment:
   For clarification, recommend renaming this to `JSON Schema`.
   ```suggestion
           .Builder().name("JSON Schema")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")

Review Comment:
   ```suggestion
           .description("The content of a JSON Schema")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {

Review Comment:
   ```suggestion
           if (flowFile == null) {
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }

Review Comment:
   It would be more efficient to set these collections in a static initializer, instead of this `init()` method, which is not commonly used.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);
+                session.transfer(flowFile, REL_INVALID);
+            } else {
+                // Schema check passed
+                this.getLogger().debug("Successfully validated {} against schema; routing to 'valid'", flowFile);

Review Comment:
   ```suggestion
                   this.getLogger().debug("JSON {} valid", flowFile);
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);

Review Comment:
   Recommend adjusting the message to include the number of errors.
   ```suggestion
                   this.getLogger().info("JSON {} invalid: Validation Errors [{}]", flowFile, errors.size());
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;

Review Comment:
   This property declarations should be moved up, prior to method declarations.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)

Review Comment:
   Recommend removing support for Expression Language because the Variable Registry is not widely used and will likely be removed in the next major version of NiFi. Removing Expression Language support also opens the possibility for caching the Schema.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);
+                session.transfer(flowFile, REL_INVALID);
+            } else {
+                // Schema check passed
+                this.getLogger().debug("Successfully validated {} against schema; routing to 'valid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_VALID);

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java:
##########
@@ -0,0 +1,202 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.ProcessorInitializationContext;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.util.JsonValidator;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import com.networknt.schema.JsonSchema;
+import com.networknt.schema.JsonSchemaFactory;
+import com.networknt.schema.ValidationMessage;
+import com.networknt.schema.SpecVersion.VersionFlag;
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"JSON", "schema", "validation"})
+@WritesAttributes({
+    @WritesAttribute(attribute = "validatejson.invalid.error", description = "If the flow file is routed to the invalid relationship "
+            + "the attribute will contain the error message resulting from the validation failure.")
+})
+@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file")
+public class ValidateJson extends AbstractProcessor {
+
+    public static final String ERROR_ATTRIBUTE_KEY = "validatejson.invalid.error";
+
+    public static final AllowableValue SCHEMA_VERSION_4 = new AllowableValue("V4");
+    public static final AllowableValue SCHEMA_VERSION_6 = new AllowableValue("V6");
+    public static final AllowableValue SCHEMA_VERSION_7 = new AllowableValue("V7");
+    public static final AllowableValue SCHEMA_VERSION_V201909 = new AllowableValue("V201909");
+
+    public static final PropertyDescriptor SCHEMA_VERSION = new PropertyDescriptor
+        .Builder().name("SCHEMA_VERSION")
+        .displayName("Schema Version")
+        .description("The JSON schema specification")
+        .required(true)
+        .allowableValues(SCHEMA_VERSION_4, SCHEMA_VERSION_6, SCHEMA_VERSION_7, SCHEMA_VERSION_V201909)
+        .defaultValue(SCHEMA_VERSION_V201909.getValue())
+        .build();
+
+    public static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor
+        .Builder().name("SCHEMA_TEXT")
+        .displayName("Schema Text")
+        .description("The text of a JSON schema")
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .addValidator(JsonValidator.INSTANCE)
+        .build();
+
+    public static final Relationship REL_VALID = new Relationship.Builder()
+        .name("valid")
+        .description("FlowFiles that are successfully validated against the schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_INVALID = new Relationship.Builder()
+        .name("invalid")
+        .description("FlowFiles that are not valid according to the specified schema are routed to this relationship")
+        .build();
+
+    public static final Relationship REL_FAILURE = new Relationship.Builder()
+        .name("failure")
+        .description("FlowFiles that cannot be read as JSON are routed to this relationship")
+        .build();
+
+    private List<PropertyDescriptor> descriptors;
+
+    private Set<Relationship> relationships;
+
+    private List<AllowableValue> allowableValues;
+
+    @Override
+    protected void init(final ProcessorInitializationContext context) {
+        final List<PropertyDescriptor> descriptors = new ArrayList<PropertyDescriptor>();
+        descriptors.add(SCHEMA_TEXT);
+        descriptors.add(SCHEMA_VERSION);
+        this.descriptors = Collections.unmodifiableList(descriptors);
+
+        final Set<Relationship> relationships = new HashSet<Relationship>();
+        relationships.add(REL_VALID);
+        relationships.add(REL_INVALID);
+        relationships.add(REL_FAILURE);
+        this.relationships = Collections.unmodifiableSet(relationships);
+
+        final List<AllowableValue> allowableValues = new ArrayList<AllowableValue>();
+        allowableValues.add(SCHEMA_VERSION_4);
+        allowableValues.add(SCHEMA_VERSION_6);
+        allowableValues.add(SCHEMA_VERSION_7);
+        allowableValues.add(SCHEMA_VERSION_V201909);
+        this.allowableValues = Collections.unmodifiableList(allowableValues);
+
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return this.relationships;
+    }
+
+    @Override
+    public final List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return descriptors;
+    }
+
+    private ObjectMapper mapper = new ObjectMapper();
+    private VersionFlag schemaVersion;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        // Set JSON schema version to use from processor property
+        this.schemaVersion = VersionFlag.V201909;
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_4.getValue()) {
+            this.schemaVersion = VersionFlag.V4;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_6.getValue()) {
+            this.schemaVersion = VersionFlag.V6;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_7.getValue()) {
+            this.schemaVersion = VersionFlag.V7;
+        }
+        if (context.getProperty(SCHEMA_VERSION).getValue() == SCHEMA_VERSION_V201909.getValue()) {
+            this.schemaVersion = VersionFlag.V201909;
+        }
+    }
+
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        FlowFile flowFile = session.get();
+        if ( flowFile == null ) {
+            return;
+        }
+
+        try (InputStream in = session.read(flowFile)) {
+            // Read in flowFile inputstream, and validate against schema
+            JsonNode node = mapper.readTree(in);
+            String schemaText = context.getProperty(SCHEMA_TEXT).evaluateAttributeExpressions().getValue();
+            JsonSchemaFactory factory = JsonSchemaFactory.getInstance(schemaVersion);
+            JsonSchema schema = factory.getSchema(schemaText);
+            Set<ValidationMessage> errors = schema.validate(node);
+
+            if (errors.size() > 0) {
+                // Schema checks failed
+                flowFile = session.putAttribute(flowFile, ERROR_ATTRIBUTE_KEY, errors.toString());
+                this.getLogger().info("Found {} to be invalid when validated against schema; routing to 'invalid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_INVALID);
+                session.transfer(flowFile, REL_INVALID);
+            } else {
+                // Schema check passed
+                this.getLogger().debug("Successfully validated {} against schema; routing to 'valid'", flowFile);
+                session.getProvenanceReporter().route(flowFile, REL_VALID);
+                session.transfer(flowFile, REL_VALID);
+            }
+
+        } catch (IOException ioe) {
+            // Failed to read flowFile

Review Comment:
   ```suggestion
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestValidateJson.java:
##########
@@ -0,0 +1,136 @@
+/*
+ * 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.processors.standard;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;

Review Comment:
   New unit tests should be implemented using JUnit 5, so these references should be replaced with the corresponding JUnit Jupiter references.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org