[ 
https://issues.apache.org/jira/browse/NIFI-361?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15246341#comment-15246341
 ] 

ASF GitHub Bot commented on NIFI-361:
-------------------------------------

Github user YolandaMDavis commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/354#discussion_r60115891
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformJSON.java
 ---
    @@ -0,0 +1,212 @@
    +/*
    + * 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.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +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.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.AllowableValue;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.exception.ProcessException;
    +import org.apache.nifi.processor.io.StreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.stream.io.BufferedInputStream;
    +import org.apache.nifi.util.StopWatch;
    +
    +import com.bazaarvoice.jolt.Shiftr;
    +import com.bazaarvoice.jolt.Removr;
    +import com.bazaarvoice.jolt.Chainr;
    +import com.bazaarvoice.jolt.Defaultr;
    +import com.bazaarvoice.jolt.Transform;
    +import com.bazaarvoice.jolt.JsonUtils;
    +
    +@EventDriven
    +@SideEffectFree
    +@SupportsBatching
    +@Tags({"json", "jolt", "transform", "shiftr", "chainr", "defaultr", 
"removr"})
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Applies a list of JOLT specifications to the 
flowfile JSON payload. A new FlowFile is created "
    +        + "with transformed content and is routed to the 'success' 
relationship. If the JSON transform "
    +        + "fails, the original FlowFile is routed to the 'failure' 
relationship")
    +public class TransformJSON extends AbstractProcessor {
    +
    +    public static final AllowableValue SHIFTR = new 
AllowableValue("Shift", "Shift Transform DSL", "This JOLT transformation will 
shift input JSON/data to create the output JSON/data.");
    +    public static final AllowableValue CHAINR = new 
AllowableValue("Chain", "Chain Transform DSL", "Execute list of JOLT 
transformations.");
    +    public static final AllowableValue DEFAULTR = new 
AllowableValue("Default", "Default Transform DSL", " This JOLT transformation 
will apply default values to the output JSON/data.");
    +    public static final AllowableValue REMOVR = new 
AllowableValue("Remove", "Remove Transform DSL", " This JOLT transformation 
will apply default values to the output JSON/data.");
    +
    +    public static final PropertyDescriptor JOLT_SPEC = new 
PropertyDescriptor.Builder()
    +            .name("Jolt Specification")
    +            .description("Jolt Specification for transform of JSON data.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .addValidator(new JOLTSpecValidator())
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor JOLT_TRANSFORM = new 
PropertyDescriptor.Builder()
    +            .name("Jolt Transformation")
    +            .description("Specifies the Jolt Transformation that should be 
used with the provided specification.")
    +            .required(true)
    +            .allowableValues(SHIFTR, CHAINR, DEFAULTR, REMOVR)
    +            .defaultValue(CHAINR.getValue())
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
    +            .name("success")
    +            .description("The FlowFile with transformed content will be 
routed to this relationship")
    +            .build();
    +    public static final Relationship REL_FAILURE = new 
Relationship.Builder()
    +            .name("failure")
    +            .description("If a FlowFile fails processing for any reason 
(for example, the FlowFile is not valid JSON), it will be routed to this 
relationship")
    +            .build();
    +
    +    private List<PropertyDescriptor> properties;
    +    private Set<Relationship> relationships;
    +    private Transform transform;
    +
    +
    +    @Override
    +    protected void init(ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> properties = new ArrayList<>();
    +        properties.add(JOLT_TRANSFORM);
    +        properties.add(JOLT_SPEC);
    +        this.properties = Collections.unmodifiableList(properties);
    +
    +        final Set<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return properties;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, ProcessSession 
session) throws ProcessException {
    +
    +        final FlowFile original = session.get();
    +        if (original == null) {
    +            return;
    +        }
    +
    +        final ProcessorLog logger = getLogger();
    +        final StopWatch stopWatch = new StopWatch(true);
    +
    +        try {
    +
    +            FlowFile transformed = session.write(original, new 
StreamCallback() {
    +                @Override
    +                public void process(final InputStream rawIn, final 
OutputStream out) throws IOException {
    +
    +                    try (final InputStream in = new 
BufferedInputStream(rawIn)) {
    +                        Object inputJson = JsonUtils.jsonToObject(in);
    +                        Object transformedJson = 
transform.transform(inputJson);
    +                        
out.write(JsonUtils.toJsonString(transformedJson).getBytes());
    +                    } catch (final Exception e) {
    +                        throw new IOException(e);
    --- End diff --
    
    Hi Oleg, based on your comments I used IntelliJ's coverage tools and 
confirmed this section of code in question was reached.  The test 
TestTransformJSON.testInvalidFlowFileContent triggers the exception. If the 
flow file content isn't json the utility class I'm using to parse (which is 
backed by Jackson) internally catches a JsonParseException and rethrows it as a 
RuntimeException. If it allowed the JsonParseException through it would have 
been fine, since it extends the IOException. Not quite sure why they wrapped it 
in a RuntimeException. 
    Anyway I did confirm that not having the catch block misses the 
ProcessException altogether and my test fails, since it's checking that flow 
files were routed to the failure relationship.
    Also I did add a couple of tests to ensure 100% coverage.
    
    Will commit and push changes shortly for review.


> Create Processors to mutate JSON data
> -------------------------------------
>
>                 Key: NIFI-361
>                 URL: https://issues.apache.org/jira/browse/NIFI-361
>             Project: Apache NiFi
>          Issue Type: New Feature
>            Reporter: Aldrin Piri
>            Assignee: Oleg Zhurakousky
>            Priority: Minor
>             Fix For: 1.0.0, 0.7.0
>
>
> Creating a separate issue to track these as a pull request has been submitted 
> for related issue NIFI-356.
> Also backed by JsonPath, processors should facilitate through specification 
> of user-defined properties:
> * Add - identify path and add key/value pair
> ** Handle if the path is an array, this would ignore the name specified and 
> just add the value to the collection
> * Remove - delete the element at the specified path
> * Update - change the value for the given path to a provided value
> Need to determine if objects/arrays make sense for values or if they are 
> needed.
> While it would be nice to be able to execute several operations per processor 
> instance, it may be hard to capture all the relevant information needed for 
> multiple operations in one processor configuration in a user friendly context.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to