If each record has distinct logic, you could also use a PartitionRecord [1] 
processor to at least organize similar records in output flowfiles, and then 
operate on each “group” with a specific processor. For example, if the logic 
for Type A, Type B, and Type C records are very different, you could create a 
record-oriented processor for each, and do something like the following:

Input:

id, type, name
1, A, Ameer
2, B, Bryan
3, A, Andy
4, C, Christine
5, C, Charlie

Your PartitionRecord processors would use a RecordPath [2] expression over 
“/type" and have an output relationship for A and “other”, and then repeat with 
B and C. Each of those relationships could feed to a ProcessTypeX custom 
processor wrapping the transformation logic you’ve already written.  

[1] 
https://nifi.apache.org/docs/nifi-docs/components/org.apache.nifi/nifi-standard-nar/1.8.0/org.apache.nifi.processors.standard.PartitionRecord/index.html
 
<https://nifi.apache.org/docs/nifi-docs/components/org.apache.nifi/nifi-standard-nar/1.8.0/org.apache.nifi.processors.standard.PartitionRecord/index.html>
[2] 
https://nifi.apache.org/docs/nifi-docs/html/record-path-guide.html#structure 
<https://nifi.apache.org/docs/nifi-docs/html/record-path-guide.html#structure>


Andy LoPresto
[email protected]
[email protected]
PGP Fingerprint: 70EC B3E5 98A6 5A3F D3C4  BACE 3C6E F65B 2F7D EF69

> On Nov 2, 2018, at 7:21 AM, Ameer Mawia <[email protected]> wrote:
> 
> Inline.
> 
> On Thu, Nov 1, 2018 at 1:40 PM Bryan Bende <[email protected] 
> <mailto:[email protected]>> wrote:
> How big are the initial CSV files?
> 
> If they are large, like millions of lines, or even hundreds of
> thousands, then it will be ideal if you can avoid the line-by-line
> split, and instead process the lines in place.
> 
> Not million. But definitely ranging from 10s to 100s of thousand.
>  
> This is one of the benefits of the record processors. For example,
> with UpdateRecord you can read in a large CSV line by line, apply an
> update to each line, and write it back out. So you only ever have one
> flow file.
> 
> Agreed.
>  
> It sounds like you may have a significant amount of custom logic so
> you may need a custom processor,
> Yes. Each record has its own logic. On top of that some time multiple data 
> source are referred to determine the final value of the output field. 
> but you can still take this approach
> of reading a single flow file line by line, and writie out the results
> line by line (try to avoid reading the entire content into memory at
> one time). 
> That what I am trying.
>  
> On Thu, Nov 1, 2018 at 1:22 PM Ameer Mawia <[email protected] 
> <mailto:[email protected]>> wrote:
> >
> > Thanks for the input folks.
> >
> > I had this impression that for actual processing of the data :
> >
> > we may have to put in place a custom processor which will have the 
> > transformation framework logic in it.
> > Or we can use ExcecuteProcess processor to trigger an external 
> > process(which will be this transformation logic) and route back the output 
> > in the NIFI.
> >
> > Our flow inside the framework generally looks like this:
> >
> > Split the CSV file line by line.
> > For each line Split it in array of string.
> > For each record in the array determine its invoke it transformation method.
> > Transformation Method contains the transformation logic. This logic can be 
> > pretty intensive like:
> >
> > searching for hundreds of different pattern.
> > lookup against hundreds of configured string constants.
> > Appending/Prepending/Trimming/Padding...
> >
> > Finally map the each record into an output csv format.
> >
> > So far we have been trying to see if SplitRecord, UpdateRecord, 
> > ExtractText, etc can come in handy?
> >
> > Thanks,
> >
> > On Thu, Nov 1, 2018 at 12:39 PM Mike Thomsen <[email protected] 
> > <mailto:[email protected]>> wrote:
> >>
> >> Ameer,
> >>
> >> Depending on how you implemented the custom framework, you may be able to 
> >> easily drop it in place into a custom NiFi processor. Without knowing much 
> >> about your implementation details, if you can act on Java streams, 
> >> Strings, byte arrays and things like that it will probably be very 
> >> straight forward to drop in place.
> >>
> >> This is a really simple of how you could bring it in depending on how 
> >> encapsulated your business logic is:
> >>
> >> @Override
> >> public void onTrigger(ProcessContext context, ProcessSession session) 
> >> throws ProcessException {
> >>     FlowFile input = session.get();
> >>     if (input == null) {
> >>         return;
> >>     }
> >>
> >>     FlowFile output = session.create(input);
> >>     try (InputStream is = session.read(input);
> >>         OutputStream os = session.write(output)
> >>     ) {
> >>         transformerPojo.transform(is, os);
> >>
> >>         is.close();
> >>         os.close();
> >>
> >>         session.transfer(input, REL_ORIGINAL); //If you created an 
> >> "original relationship"
> >>         session.transfer(output, REL_SUCCESS);
> >>     } catch (Exception ex) {
> >>         session.remove(output);
> >>         session.transfer(input, REL_FAILURE);
> >>     }
> >> }
> >>
> >> That's the general idea, and that approach can scale to your disk space 
> >> limits. Hope that helps put it into perspective.
> >>
> >> Mike
> >>
> >> On Thu, Nov 1, 2018 at 10:16 AM Nathan Gough <[email protected] 
> >> <mailto:[email protected]>> wrote:
> >>>
> >>> Hi Ameer,
> >>>
> >>> This blog by Mark Payne describes how to manipulate record based data 
> >>> like CSV using schemas: 
> >>> https://blogs.apache.org/nifi/entry/record-oriented-data-with-nifi 
> >>> <https://blogs.apache.org/nifi/entry/record-oriented-data-with-nifi>. 
> >>> This would probably be the most efficient method. And another here: 
> >>> https://bryanbende.com/development/2017/06/20/apache-nifi-records-and-schema-registries
> >>>  
> >>> <https://bryanbende.com/development/2017/06/20/apache-nifi-records-and-schema-registries>.
> >>>
> >>> An alternative option would be to port your custom java code into your 
> >>> own NiFi processor:
> >>> https://medium.com/hashmapinc/creating-custom-processors-and-controllers-in-apache-nifi-e14148740ea
> >>>  
> >>> <https://medium.com/hashmapinc/creating-custom-processors-and-controllers-in-apache-nifi-e14148740ea>
> >>>  under 'Steps for Creating a Custom Apache NiFi Processor'
> >>> https://nifi.apache.org/developer-guide.html 
> >>> <https://nifi.apache.org/developer-guide.html>
> >>>
> >>> Nathan
> >>>
> >>> On 10/31/18, 5:02 PM, "Ameer Mawia" <[email protected] 
> >>> <mailto:[email protected]>> wrote:
> >>>
> >>>     We have a use case where we take data from a source(text data in csv
> >>>     format), do transformation and manipulation of textual record, and 
> >>> output
> >>>     the data in another (csv)format. This is being done by a Java based 
> >>> custom
> >>>     framework, written specifically for this *transformation* piece.
> >>>
> >>>     Recently as Apache NIFI is being adopted at enterprise level by the
> >>>     organisation, we have been asked to try *Apache NIFI* and see if can 
> >>> use
> >>>     that as a replacement to this custom tool?
> >>>
> >>>     *My question is*:
> >>>
> >>>        - How much leverage does *Apache NIFI *provides on the flowfile 
> >>> *content
> >>>        *manipulation?
> >>>
> >>>     I understand *NIFI *is good for creating data flow pipeline, but is 
> >>> it good
> >>>     for *extensive TEXT Transformation* as well?   So far I have not found
> >>>     obvious way to achieve that.
> >>>
> >>>     Appreciate the feedback.
> >>>
> >>>     Thanks,
> >>>
> >>>     --
> >>>     http://ca.linkedin.com/in/ameermawia 
> >>> <http://ca.linkedin.com/in/ameermawia>
> >>>     Toronto, ON
> >>>
> >>>
> >>>
> >
> >
> > --
> > http://ca.linkedin.com/in/ameermawia <http://ca.linkedin.com/in/ameermawia>
> > Toronto, ON
> >
> 
> 
> -- 
> http://ca.linkedin.com/in/ameermawia <http://ca.linkedin.com/in/ameermawia>
> Toronto, ON

Reply via email to