[ https://issues.apache.org/jira/browse/NIFI-1893?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15507083#comment-15507083 ]
ASF GitHub Bot commented on NIFI-1893: -------------------------------------- Github user pvillard31 commented on a diff in the pull request: https://github.com/apache/nifi/pull/1037#discussion_r79655327 --- Diff: nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateJson.java --- @@ -0,0 +1,159 @@ +/* + * 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.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.commons.io.IOUtils; +import org.apache.nifi.annotation.behavior.EventDriven; +import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.SideEffectFree; +import org.apache.nifi.annotation.behavior.SupportsBatching; +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.components.PropertyDescriptor; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +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.io.InputStreamCallback; +import org.apache.nifi.processor.util.StandardValidators; + +import org.everit.json.schema.Schema; +import org.everit.json.schema.ValidationException; +import org.everit.json.schema.loader.SchemaLoader; +import org.json.JSONArray; +import org.json.JSONObject; +import org.json.JSONTokener; + + +@EventDriven +@SideEffectFree +@SupportsBatching +@InputRequirement(Requirement.INPUT_REQUIRED) +@Tags({"json", "schema", "validation"}) +@CapabilityDescription("Validates the contents of FlowFiles against a user-specified JSON Schema file") +public class ValidateJson extends AbstractProcessor { + + public static final PropertyDescriptor SCHEMA_FILE = new PropertyDescriptor.Builder() + .name("Schema File") + .description("The path to the Schema file that is to be used for validation") + .required(true) + .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .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(); + + private List<PropertyDescriptor> properties; + private Set<Relationship> relationships; + private final AtomicReference<Schema> schemaRef = new AtomicReference<>(); + + @Override + protected void init(final ProcessorInitializationContext context) { + final List<PropertyDescriptor> properties = new ArrayList<>(); + properties.add(SCHEMA_FILE); + this.properties = Collections.unmodifiableList(properties); + + final Set<Relationship> relationships = new HashSet<>(); + relationships.add(REL_VALID); + relationships.add(REL_INVALID); + this.relationships = Collections.unmodifiableSet(relationships); + } + + @Override + public Set<Relationship> getRelationships() { + return relationships; + } + + @Override + protected List<PropertyDescriptor> getSupportedPropertyDescriptors() { + return properties; + } + + @OnScheduled + public void parseSchema(final ProcessContext context) throws IOException { + try(FileInputStream inputStream = new FileInputStream(new File(context.getProperty(SCHEMA_FILE).getValue()))) { + JSONTokener jsonTokener = new JSONTokener(inputStream); + JSONObject jsonObject = new JSONObject(jsonTokener); + Schema schema = SchemaLoader.load(jsonObject); + this.schemaRef.set(schema); + } + } + + @Override + public void onTrigger(final ProcessContext context, final ProcessSession session) { + final List<FlowFile> flowFiles = session.get(50); --- End diff -- I would suggest not doing an internal batching here. Some older processors did this because "@SupportsBatching" wasn't created yet. You already have this annotation so this will allow the user to set the batching duration instead of hiding inside the implementation. > Add processor for validating JSON > --------------------------------- > > Key: NIFI-1893 > URL: https://issues.apache.org/jira/browse/NIFI-1893 > Project: Apache NiFi > Issue Type: New Feature > Reporter: Matt Burgess > > NiFi has a ValidateXml processor to validate incoming XML files against a > schema. It would be good to have one to validate JSON files as well. > For example, an input JSON of: > { > name: "Test", > timestamp: 1463499695, > tags: { > "host": "Test_1", > "ip" : "1.1.1.1" > }, > fields: { > "cpu": 10.2, > "load": 15.6 > } > } > Could be validated successfully against the following "schema": > { > "type": "object", > "required": ["name", "tags", "timestamp", "fields"], > "properties": { > "name": {"type": "string"}, > "timestamp": {"type": "integer"}, > "tags": {"type": "object", "items": {"type": "string"}}, > "fields": { "type": "object"} > } > } > There is at least one ASF-friendly library that could be used for > implementation: https://github.com/everit-org/json-schema -- This message was sent by Atlassian JIRA (v6.3.4#6332)