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

    https://github.com/apache/nifi/pull/1576#discussion_r110553180
  
    --- Diff: 
nifi-nar-bundles/nifi-morphlines-bundle/nifi-morphlines-processors/src/main/java/org/apache/nifi/processors/morphlines/ImplementMorphlines.java
 ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.morphlines;
    +
    +import com.google.common.base.Preconditions;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.processor.*;
    +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.io.InputStreamCallback;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.io.StreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.stream.io.StreamUtils;
    +import org.kitesdk.morphline.api.Command;
    +import org.kitesdk.morphline.api.MorphlineContext;
    +import org.kitesdk.morphline.api.Record;
    +import org.kitesdk.morphline.base.Fields;
    +
    +import org.kitesdk.morphline.base.Notifications;
    +import com.google.common.collect.ImmutableList;
    +import com.google.common.collect.ImmutableSet;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +
    +import java.io.File;
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.io.OutputStream;
    +import java.util.*;
    +import java.util.stream.*;
    +import java.util.concurrent.atomic.*;
    +import org.apache.nifi.annotation.lifecycle.OnStopped;
    +import org.apache.nifi.processor.exception.*;
    +
    +@Tags({"kitesdk", "morphlines", "ETL", "HDFS", "avro", "Solr", "HBase"})
    +@CapabilityDescription("Implements Morphlines 
(http://kitesdk.org/docs/1.1.0/morphlines/) framework, which performs in-memory 
container of transformation commands in oder to perform tasks such as loading, 
parsing, transforming, or otherwise processing a single record.")
    +public class MorphlinesProcessor extends AbstractProcessor {
    +
    +    private Command morphline;
    +    private volatile Record record = new Record();
    +    private volatile Collector collector = new Collector();
    +
    +    public static final PropertyDescriptor MORPHLINES_ID = new 
PropertyDescriptor
    +            .Builder().name("Morphlines ID")
    +            .description("Identifier of the morphlines context")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor MORPHLINES_FILE = new 
PropertyDescriptor
    +            .Builder().name("Morphlines File")
    +            .description("File for the morphlines context")
    +            .required(true)
    +            .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    public static final PropertyDescriptor MORPHLINES_OUTPUT_FIELD = new 
PropertyDescriptor
    +            .Builder().name("Morphlines output field")
    +            .description("Field name of output in Morphlines. Default is 
'_attachment_body'.")
    +            .required(false)
    +       .expressionLanguageSupported(true)
    +            .defaultValue("_attachment_body")
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
    +            .name("success")
    +            .description("Relationship for success.")
    +            .build();
    +
    +    public static final Relationship REL_FAILURE = new 
Relationship.Builder()
    +            .name("failure")
    +            .description("Relationship for failure of morphlines.")
    +            .build();
    +
    +    private static final List<PropertyDescriptor> PROPERTIES = 
ImmutableList.<PropertyDescriptor>builder()
    +            .add(MORPHLINES_FILE)
    +            .add(MORPHLINES_ID)
    +            .add(MORPHLINES_OUTPUT_FIELD)
    +            .build();
    +
    +    private static final Set<Relationship> RELATIONSHIPS = 
ImmutableSet.<Relationship>builder()
    +            .add(REL_SUCCESS)
    +            .add(REL_FAILURE)
    +            .build();
    +
    +
    +    private File morphLinesFile;
    +    private String morphLinesId;
    +    private String morphlinesOutputField;
    +    private PropertyValue morphLinesFileProperty;
    +    private PropertyValue morphLinesIdProperty;
    +    private PropertyValue morphLinesOutputField;
    +    private MorphlineContext morphlineContext;
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return RELATIONSHIPS;
    +    }
    +
    +    @Override
    +    public final List<PropertyDescriptor> 
getSupportedPropertyDescriptors() {
    +        return PROPERTIES;
    +    }
    +
    +    @OnScheduled
    +    public void onScheduled(ProcessContext context) throws Exception {
    +   morphlinesFileProperty = context.getProperty(MORPHLINES_FILE);
    +   morphlinesIdProperty = context.getProperty(MORPHLINES_ID);
    +   morphlinesOutputFieldProperty = 
context.getProperty(MORPHLINES_OUTPUT_FIELD);
    +        morphlineContext = new MorphlineContext.Builder().build();
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final 
ProcessSession session) throws ProcessException {
    +        FlowFile flowFile = session.get();
    +        if ( flowFile == null ) {
    +            return;
    +        }
    +
    +   morphLinesFile = new 
File(morphlinesFileProperty.evaluateAttributeExpressions(flowfile).getValue());
    +   morphLinesId = 
morphlinesIdProperty.evaluateAttributeExpressions(flowfile).getValue();
    +   morphlinesOutputField = 
morphlinesOutputFieldProperty.evaluateAttributeExpressions(flowfile).getValue();
    +
    +        final AtomicLong written = new AtomicLong(0L);
    +        final byte[] value = new byte[(int) flowFile.getSize()];
    +
    +        try{
    +            flowFile = session.write(flowFile, new StreamCallback() {
    +                @Override
    +                public void process(InputStream in, OutputStream out) 
throws IOException {
    +                    StreamUtils.fillBuffer(in, value);
    +                    Record record = new Record();
    +                    record.put(Fields.ATTACHMENT_BODY, value);
    +                    Collector collectorRecord = new Collector();
    +                    morphline = new 
org.kitesdk.morphline.base.Compiler().compile(morphLinesFile, morphLinesId, 
morphlineContext, collectorRecord);
    --- End diff --
    
    You said before that you couldn't move this out of the onTrigger due to 
needing the collectorRecord to create and compile it.
    
    According to the Cloudera intro documentation "Example Driver Program". It 
creates the compiler then processes each record. Can't we do the same here? 
Relevant section below:
    
    ```
      MorphlineContext context = new MorphlineContext.Builder().build();
      Command morphline = new Compiler().compile(configFile, null, context, 
null);
    
      // process each input data file
      Notifications.notifyBeginTransaction(morphline);
      for (int i = 1; i < args.length; i++) {
        InputStream in = new FileInputStream(new File(args[i]));
        Record record = new Record();
        record.put(Fields.ATTACHMENT_BODY, in);
        morphline.process(record);
        in.close();
      }
    ```
    
    
    
    [1] 
http://blog.cloudera.com/blog/2013/07/morphlines-the-easy-way-to-build-and-integrate-etl-apps-for-apache-hadoop/


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

Reply via email to