You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by Uwe Geercken <uw...@web.de> on 2016/03/18 16:06:05 UTC

Processor: User friendly vs system friendly design

Hello,

my first mailing here. I am a Java developer, using Apache Velocity, Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very much a data guy.

I have used Nifi for a while now and started yesterday of coding my first processor. I basically do it to widen my knowledge and learn something new.

I started with the idea of combining Apache Velocity - a template engine - with Nifi. So in comes a CSV file, it gets merged with a template containing formatting information and some placeholders (and some limited logic maybe) and out comes a new set of data, formatted differently. So it separates the processing logic from the formatting. One could create HTML, XML, Json or other text based formats from it. Easy to use and very efficient.

Now my question is: Should I rather implement the logic this way that I process a whole CSV file - which usually has multiple lines? That would be good for the user as he or she has to deal with only one processor doing the work. But the logic would be more specialized.

The other way around, I could code the processor to handle one row of the CSV file and the user will have to come up with a flow that divides the CSV file into multiple flowfiles before my processor can be used. That is not so specialized but it requires more preparation work from the user.

I tend to go the second way. Also because there is already a processor that will split a file into multiple flowfiles. But I wanted to hear your opinion of what is the best way to go. Do you have a recommendation for me? (Maybe the answer is to do both?!)

Thanks for sharing your thoughts.

Uwe

Re: Re: Processor: User friendly vs system friendly design

Posted by Joe Witt <jo...@gmail.com>.
This sounds pretty cool and definitely think you're both on a good
track design-wise.

On Fri, Mar 18, 2016 at 2:04 PM, Uwe Geercken <uw...@web.de> wrote:
> Adam,
>
> thanks again.
>
> I didn't know about the contributors guide - I was always looking in the docs inside Nifi and there is a reference to the developer guide only.
>
> I will try to make a good processor for velocity first. The next step would then be to include also freemarker. I will try to keep that in mind during design and coding. I don't know anything about Markdown or Asciidoc. So I will have to have a look first.
>
> Regards,
>
> Uwe
>
>
>
>> Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
>> Von: "Adam Taft" <ad...@adamtaft.com>
>> An: dev@nifi.apache.org
>> Betreff: Re: Processor: User friendly vs system friendly design
>>
>> Uwe,
>>
>> The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
>> Developer Guide has a section dealing with reading & writing flowfile
>> attributes.  Please check these out, and then if you have any specific
>> questions, please feel free to reply.
>>
>> For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
>> mentioning the new feature, and then fork the NIFI project in Github and
>> send a Pull Request referencing the ticket.  However, if you just want some
>> feedback on suitability and consideration for inclusion, using your own
>> personal Github project and sending a link would be fine.
>>
>> Having a template conversion processor would be a nice addition.  Making it
>> generic to support Velocity, FreeMarker, and others might be really nice.
>> Extra bonus points for Markdown or Asciidoc transforms as well (but these
>> might be too separate of a use case).
>>
>> Hope this helps.
>>
>> Adam
>>
>> [1]  http://nifi.apache.org/developer-guide.html
>>
>> [2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
>>
>>
>>
>>
>> On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de> wrote:
>>
>> > Adam,
>> >
>> > interesting and I agree. that sounds very good.
>> >
>> > can you give me short tip of how to access attributes from code?
>> >
>> > once I have something usable or for testing where would I publish it? just
>> > on my github site? or is there a place for sharing?
>> >
>> > greetings
>> >
>> > Uwe
>> >
>> >
>> >
>> > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
>> > Von: "Adam Taft" <ad...@adamtaft.com>
>> > An: dev@nifi.apache.org
>> > Betreff: Re: Processor: User friendly vs system friendly design
>> > I'm probably on the far end of favoring composibility and processor reuse.
>> > In this case, I would even go one step further and suggest that you're
>> > talking about three separate operations:
>> >
>> > 1. Split a multi-line CSV input file into individual single line flowfiles.
>> > 2. Read columns from a single CSV line into flowfile attributes.
>> > 3. Pass flowfile attributes into the Velocity transform processor.
>> >
>> > The point here, have you considered driving your Velocity template
>> > transform using flowfile attributes as opposed to CSV? Flowfile attributes
>> > are NIFI's lowest common data representation, many many processors create
>> > attributes which would enable your Velocity processor to be used by more
>> > than just CSV input.
>> >
>> > Adam
>> >
>> >
>> >
>> > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
>> > wrote:
>> >
>> > >
>> > > Hello,
>> > >
>> > > my first mailing here. I am a Java developer, using Apache Velocity,
>> > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very
>> > much
>> > > a data guy.
>> > >
>> > > I have used Nifi for a while now and started yesterday of coding my first
>> > > processor. I basically do it to widen my knowledge and learn something
>> > new.
>> > >
>> > > I started with the idea of combining Apache Velocity - a template engine
>> > -
>> > > with Nifi. So in comes a CSV file, it gets merged with a template
>> > > containing formatting information and some placeholders (and some limited
>> > > logic maybe) and out comes a new set of data, formatted differently. So
>> > it
>> > > separates the processing logic from the formatting. One could create
>> > HTML,
>> > > XML, Json or other text based formats from it. Easy to use and very
>> > > efficient.
>> > >
>> > > Now my question is: Should I rather implement the logic this way that I
>> > > process a whole CSV file - which usually has multiple lines? That would
>> > be
>> > > good for the user as he or she has to deal with only one processor doing
>> > > the work. But the logic would be more specialized.
>> > >
>> > > The other way around, I could code the processor to handle one row of the
>> > > CSV file and the user will have to come up with a flow that divides the
>> > CSV
>> > > file into multiple flowfiles before my processor can be used. That is not
>> > > so specialized but it requires more preparation work from the user.
>> > >
>> > > I tend to go the second way. Also because there is already a processor
>> > > that will split a file into multiple flowfiles. But I wanted to hear your
>> > > opinion of what is the best way to go. Do you have a recommendation for
>> > me?
>> > > (Maybe the answer is to do both?!)
>> > >
>> > > Thanks for sharing your thoughts.
>> > >
>> > > Uwe
>> > >
>> >
>>

Aw: Re: Processor: User friendly vs system friendly design

Posted by Uwe Geercken <uw...@web.de>.
Adam,

thanks again.

I didn't know about the contributors guide - I was always looking in the docs inside Nifi and there is a reference to the developer guide only.

I will try to make a good processor for velocity first. The next step would then be to include also freemarker. I will try to keep that in mind during design and coding. I don't know anything about Markdown or Asciidoc. So I will have to have a look first.

Regards,

Uwe



> Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> Von: "Adam Taft" <ad...@adamtaft.com>
> An: dev@nifi.apache.org
> Betreff: Re: Processor: User friendly vs system friendly design
>
> Uwe,
> 
> The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
> Developer Guide has a section dealing with reading & writing flowfile
> attributes.  Please check these out, and then if you have any specific
> questions, please feel free to reply.
> 
> For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
> mentioning the new feature, and then fork the NIFI project in Github and
> send a Pull Request referencing the ticket.  However, if you just want some
> feedback on suitability and consideration for inclusion, using your own
> personal Github project and sending a link would be fine.
> 
> Having a template conversion processor would be a nice addition.  Making it
> generic to support Velocity, FreeMarker, and others might be really nice.
> Extra bonus points for Markdown or Asciidoc transforms as well (but these
> might be too separate of a use case).
> 
> Hope this helps.
> 
> Adam
> 
> [1]  http://nifi.apache.org/developer-guide.html
> 
> [2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> 
> 
> 
> 
> On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de> wrote:
> 
> > Adam,
> >
> > interesting and I agree. that sounds very good.
> >
> > can you give me short tip of how to access attributes from code?
> >
> > once I have something usable or for testing where would I publish it? just
> > on my github site? or is there a place for sharing?
> >
> > greetings
> >
> > Uwe
> >
> >
> >
> > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > Von: "Adam Taft" <ad...@adamtaft.com>
> > An: dev@nifi.apache.org
> > Betreff: Re: Processor: User friendly vs system friendly design
> > I'm probably on the far end of favoring composibility and processor reuse.
> > In this case, I would even go one step further and suggest that you're
> > talking about three separate operations:
> >
> > 1. Split a multi-line CSV input file into individual single line flowfiles.
> > 2. Read columns from a single CSV line into flowfile attributes.
> > 3. Pass flowfile attributes into the Velocity transform processor.
> >
> > The point here, have you considered driving your Velocity template
> > transform using flowfile attributes as opposed to CSV? Flowfile attributes
> > are NIFI's lowest common data representation, many many processors create
> > attributes which would enable your Velocity processor to be used by more
> > than just CSV input.
> >
> > Adam
> >
> >
> >
> > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
> > wrote:
> >
> > >
> > > Hello,
> > >
> > > my first mailing here. I am a Java developer, using Apache Velocity,
> > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very
> > much
> > > a data guy.
> > >
> > > I have used Nifi for a while now and started yesterday of coding my first
> > > processor. I basically do it to widen my knowledge and learn something
> > new.
> > >
> > > I started with the idea of combining Apache Velocity - a template engine
> > -
> > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > containing formatting information and some placeholders (and some limited
> > > logic maybe) and out comes a new set of data, formatted differently. So
> > it
> > > separates the processing logic from the formatting. One could create
> > HTML,
> > > XML, Json or other text based formats from it. Easy to use and very
> > > efficient.
> > >
> > > Now my question is: Should I rather implement the logic this way that I
> > > process a whole CSV file - which usually has multiple lines? That would
> > be
> > > good for the user as he or she has to deal with only one processor doing
> > > the work. But the logic would be more specialized.
> > >
> > > The other way around, I could code the processor to handle one row of the
> > > CSV file and the user will have to come up with a flow that divides the
> > CSV
> > > file into multiple flowfiles before my processor can be used. That is not
> > > so specialized but it requires more preparation work from the user.
> > >
> > > I tend to go the second way. Also because there is already a processor
> > > that will split a file into multiple flowfiles. But I wanted to hear your
> > > opinion of what is the best way to go. Do you have a recommendation for
> > me?
> > > (Maybe the answer is to do both?!)
> > >
> > > Thanks for sharing your thoughts.
> > >
> > > Uwe
> > >
> >
>

Aw: Re: Re: Re: Processor: User friendly vs system friendly design

Posted by Uwe Geercken <uw...@web.de>.
Adam,

thanks for your feedback. You have some viable points here - let me answer you quickly from my train trip to work and I will have a deeper look this evening.

I would be happy to add the processors to the Nifi standard distribution. It there documentation from the project of how you specifically work with git? I know git but how does your workflow look like?

CSV Processor: 
- opencsv looks great that will be easy to do. If the processor handles a multiline file then my question would be if it should also output (additionally) a multiline flowfile vs many flowfiles?
- I agree on the header feature that makes the correlation of csv header to template placeholders easier for users
- DecimalFormat: I have a reference somewhere in the code but a more precise wording and a link would be appropriate I think
- Naming: As discussed, at this stage I am not sure how the final result will look like. E.g. if another template engine like freemaker can be used as well. The final scope will also have an influence on the naming of the pieces. I think we can discuss the details further down the road.

Velocity Processor:
- Map: if I send the map to velocity then it needs a name in the velocity context. like e.g. "mymap". in the template one would have to reference a field (e.g. column_001) like this: ${mymap.column_001}. So that is different for the user. Without the map  it would be ${column_001}.
- Regex filter: A nicety. But I thought there might be a use case where you would not want all attributes to be passed to velocity.


General:
I am still somewhat unhappy of duplicating the data into attribute values. As said this way I mix the data/content with the attributes. And by keeping the original data in the relationship "original" it's actually trippled. That could be a performance issue. WHat do you think.

My train arrives. Talk to you later.

Regards,

Uwe



> Gesendet: Montag, 21. März 2016 um 04:14 Uhr
> Von: "Adam Taft" <ad...@adamtaft.com>
> An: dev@nifi.apache.org
> Betreff: Re: Re: Re: Processor: User friendly vs system friendly design
>
> Uwe,
> 
> Great progress.
> 
> Do you intend that your processors are part of the Apache NiFi standard
> distribution?  If so, I think there is enough here to warrant switching the
> discussion over to JIRA tickets with accompanying Github activity.
> 
> Since you've split the functionality into two processors, I'd probably
> recommend creating two feature enhancement JIRA tickets, one for the CSV
> related processor and another for the Velocity processor.  If you agree,
> please open JIRA tickets at [1].
> 
> Here are some thoughts on your processors:
> 
> :: CSV Processor Comments ::
> 
> -  For the CSV processor, it's doing a very naive String#split and doesn't
> deal with quoted CSV files.  What are your thoughts of providing more
> robust CSV handling support?  Perhaps using the opencsv project? [2]
> 
> -  I think I like the ability for the CSV processor to deal with multiple
> rows.  I know I recommend you try using SplitText for this.  But it
> probably wouldn't hurt for this processor to accommodate multi-line CSV
> input files.  This way, you can throw a single line file or multi-line
> file, it will handle both.  For multiple lines, it would export a new
> flowfile for each row in the CSV input.
> 
> -  If we support multi-line CSV input files, I think it makes sense that it
> could also deal with a header line as well.  And optionally, it could
> provide prefix values extracted from the header line.  i.e. if the header
> line names columns "name", "email", "dob", the processor would export
> flowfile attributes with prefix.name, prefix.email, and prefix.dob (instead
> of using numbers).
> 
> -  I like using DecimalFormat in your solution.  Maybe we should reference
> the DecimalFormat API in your documentation.
> 
> -  We should probably consider a name with the phrase "CSV" in it, which I
> think is appropriate for the NIFI naming style.  Something like
> ConvertCSVToAttributes.  I think the NIFI community will probably help
> shape the best name.
> 
> :: Velocity Processor Comments ::
> 
> -  Is there any harm sending the entire Map<String, String> of flowfile
> attributes to Velocity?  Why did you decide to include a regex to filter
> the attributes send to the engine?  I don't disagree with this feature, but
> wondering if it's required or just a nicety.
> 
> -  There are likely some code review and naming items to talk about.  For
> example, the reference to the VelocityEngine or Template may not be safely
> published.  And the name of the processor should probably reference
> Velocity.
> 
> Thanks,
> 
> Adam
> 
> [1] https://issues.apache.org/jira/browse/NIFI
> [1] http://opencsv.sourceforge.net
> 
> 
> 
> On Sat, Mar 19, 2016 at 2:09 PM, Uwe Geercken <uw...@web.de> wrote:
> 
> > Adam,
> >
> > ok. I got it. This way one can use any flowfile attributes and merge them
> > with the template.
> >
> > I have worked all day on the two processors and it does work - very cool.
> > Will have to do more error testing though.
> >
> > So the SplitToAttribute processor does the splitting of the flowfile
> > content. It has three properties:
> >
> > Attribute prefix: how the attributes will be prefixed
> > Field seperator: used to split the content
> > Field Number format: how to format the positional number of the field
> >
> > The MergeTemplate processor merges attributes from a flowfile with the
> > Apache Velocity template. It has three properties:
> >
> > Template path: where the template is stored
> > Template name: name of the template
> > Attribute filter: which fields to make available to the template engine
> >
> > And then I have two relationships outgoing from the MergeTemplate
> > processor: One for the original content of the flowfile and one for the
> > result of the merge of the attributes with the template.
> >
> > Will do more testing. And still my additionalDetails.html documentation
> > does not work.
> >
> > Rgds,
> >
> > Uwe
> >
> >
> > > Gesendet: Freitag, 18. März 2016 um 22:41 Uhr
> > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > An: dev@nifi.apache.org
> > > Betreff: Re: Re: Processor: User friendly vs system friendly design
> > >
> > > Uwe,
> > >
> > > I'll take a look at your code sometime soon.  However, just to point you
> > in
> > > the direction, I'd suggest extracting your single line CSV data into
> > > flowfile attributes named as you've demonstrated.  i.e.  create a
> > processor
> > > which reads each CSV column as a flowfile attribute, using a configured
> > > naming convention.
> > >
> > > For example, using "column" as your prefix with your example input, you'd
> > > end up with a single flowfile with attributes like:
> > >
> > > column0 = Peterson
> > > column1 = Jenny
> > > column2 = New York
> > > column3 = USA
> > >
> > > Flowfile attributes are effectively a Map<String,String>.  So in your
> > > Velocity processor, you would pass the Map of flowfile attributes to the
> > > template engine and record the results to the flowfile content.
> > >
> > > Using SplitText seems correct up front (though like you said, you lose
> > the
> > > CSV header line).  You'd need two additional processors, from my
> > > perspective:
> > >
> > > (input) -> SplitText -> ExtractCSVColumns -> ApplyVelocityTemplate ->
> > > (output)
> > >
> > > It's the "​split row into fields and merge with template" that we would
> > > want to separate into two processors instead of one.
> > >
> > > You're very much on the right track, I believe.  If the above doesn't
> > help,
> > > I'll try and jump in on a code example when I can.
> > >
> > > Adam
> > >
> > >
> > > On Fri, Mar 18, 2016 at 5:04 PM, Uwe Geercken <uw...@web.de>
> > wrote:
> > >
> > > > Adam,
> > > >
> > > > I don't see an obvious way for your suggestion of "Read columns from a
> > > > single CSV line into flowfile attributes." - I would need your advice
> > how I
> > > > can achieve it.
> > > >
> > > > Thinking about it in more detail, I have following issues:
> > > > - the incomming flowfile may have many columns. so adding the columns
> > > > manually as attributes with UpdateAttributes is not feasible
> > > > - I have setup a flow where I use SplitText to divide the flowfile into
> > > > multiple flowfiles, so there won't be a header row I can use to get the
> > > > column names. So I think I can only use abstract column names plus a
> > > > running number. e.g. column0, column1, etc.
> > > >
> > > > So for the moment I have coded the processor like described below. At
> > the
> > > > moment I am still "thinking in CSV" but I will check it with other
> > formats
> > > > later. The user can steer follwoing settings: path where the template
> > is
> > > > stored, name of the template file, the label for the columns (I call it
> > > > prefix) and the separator based on which the split of the row is done.
> > > >
> > > > Example Flowfile content (user has chosen "comma" as separator:
> > > >
> > > > Peterson, Jenny, New York, USA
> > > >
> > > > Example template (user has chosen "column" as the prefix):
> > > >
> > > > {
> > > >         "name": "$column0",
> > > >         "first": "$column1",
> > > >         "city": "$column2",
> > > >         "country": "$column3"
> > > > }
> > > >
> > > > Example flow:
> > > >
> > > > GetFile: Get CSV File >> SplitText : split into multiple flowfiles, one
> > > > per row >> TemplateProcessor:
> > > > ​​
> > > > split row into fields and merge with template >> MergeContent: merge
> > > > flowfiles into one >> PutFile: put the file to the filesystem
> > > >
> > > > Example result:
> > > >
> > > > {
> > > >         "name": "Peterson",
> > > >         "first": "Jenny",
> > > >         "city": "New York",
> > > >         "country": "USA"
> > > >  }
> > > >
> > > > I will test the processor now for larger files, empty files and other
> > > > exceptions. If you are interested the code is here:
> > > >
> > > > https://github.com/uwegeercken/nifi_processors
> > > >
> > > > Greetings,
> > > >
> > > > Uwe
> > > >
> > > >
> > > >
> > > > > Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> > > > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > > > An: dev@nifi.apache.org
> > > > > Betreff: Re: Processor: User friendly vs system friendly design
> > > > >
> > > > > Uwe,
> > > > >
> > > > > The Developer Guide[1] and Contributor Guide[2] are pretty solid.
> > The
> > > > > Developer Guide has a section dealing with reading & writing flowfile
> > > > > attributes.  Please check these out, and then if you have any
> > specific
> > > > > questions, please feel free to reply.
> > > > >
> > > > > For inclusion in NIFI directly, you'd want to create a NIFI Jira
> > ticket
> > > > > mentioning the new feature, and then fork the NIFI project in Github
> > and
> > > > > send a Pull Request referencing the ticket.  However, if you just
> > want
> > > > some
> > > > > feedback on suitability and consideration for inclusion, using your
> > own
> > > > > personal Github project and sending a link would be fine.
> > > > >
> > > > > Having a template conversion processor would be a nice addition.
> > Making
> > > > it
> > > > > generic to support Velocity, FreeMarker, and others might be really
> > nice.
> > > > > Extra bonus points for Markdown or Asciidoc transforms as well (but
> > these
> > > > > might be too separate of a use case).
> > > > >
> > > > > Hope this helps.
> > > > >
> > > > > Adam
> > > > >
> > > > > [1]  http://nifi.apache.org/developer-guide.html
> > > > >
> > > > > [2]
> > https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> > > > >
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de>
> > > > wrote:
> > > > >
> > > > > > Adam,
> > > > > >
> > > > > > interesting and I agree. that sounds very good.
> > > > > >
> > > > > > can you give me short tip of how to access attributes from code?
> > > > > >
> > > > > > once I have something usable or for testing where would I publish
> > it?
> > > > just
> > > > > > on my github site? or is there a place for sharing?
> > > > > >
> > > > > > greetings
> > > > > >
> > > > > > Uwe
> > > > > >
> > > > > >
> > > > > >
> > > > > > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > > > > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > > > > An: dev@nifi.apache.org
> > > > > > Betreff: Re: Processor: User friendly vs system friendly design
> > > > > > I'm probably on the far end of favoring composibility and processor
> > > > reuse.
> > > > > > In this case, I would even go one step further and suggest that
> > you're
> > > > > > talking about three separate operations:
> > > > > >
> > > > > > 1. Split a multi-line CSV input file into individual single line
> > > > flowfiles.
> > > > > > 2. Read columns from a single CSV line into flowfile attributes.
> > > > > > 3. Pass flowfile attributes into the Velocity transform processor.
> > > > > >
> > > > > > The point here, have you considered driving your Velocity template
> > > > > > transform using flowfile attributes as opposed to CSV? Flowfile
> > > > attributes
> > > > > > are NIFI's lowest common data representation, many many processors
> > > > create
> > > > > > attributes which would enable your Velocity processor to be used by
> > > > more
> > > > > > than just CSV input.
> > > > > >
> > > > > > Adam
> > > > > >
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <
> > uwe.geercken@web.de>
> > > > > > wrote:
> > > > > >
> > > > > > >
> > > > > > > Hello,
> > > > > > >
> > > > > > > my first mailing here. I am a Java developer, using Apache
> > Velocity,
> > > > > > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am
> > > > very
> > > > > > much
> > > > > > > a data guy.
> > > > > > >
> > > > > > > I have used Nifi for a while now and started yesterday of coding
> > my
> > > > first
> > > > > > > processor. I basically do it to widen my knowledge and learn
> > > > something
> > > > > > new.
> > > > > > >
> > > > > > > I started with the idea of combining Apache Velocity - a template
> > > > engine
> > > > > > -
> > > > > > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > > > > > containing formatting information and some placeholders (and some
> > > > limited
> > > > > > > logic maybe) and out comes a new set of data, formatted
> > differently.
> > > > So
> > > > > > it
> > > > > > > separates the processing logic from the formatting. One could
> > create
> > > > > > HTML,
> > > > > > > XML, Json or other text based formats from it. Easy to use and
> > very
> > > > > > > efficient.
> > > > > > >
> > > > > > > Now my question is: Should I rather implement the logic this way
> > > > that I
> > > > > > > process a whole CSV file - which usually has multiple lines? That
> > > > would
> > > > > > be
> > > > > > > good for the user as he or she has to deal with only one
> > processor
> > > > doing
> > > > > > > the work. But the logic would be more specialized.
> > > > > > >
> > > > > > > The other way around, I could code the processor to handle one
> > row
> > > > of the
> > > > > > > CSV file and the user will have to come up with a flow that
> > divides
> > > > the
> > > > > > CSV
> > > > > > > file into multiple flowfiles before my processor can be used.
> > That
> > > > is not
> > > > > > > so specialized but it requires more preparation work from the
> > user.
> > > > > > >
> > > > > > > I tend to go the second way. Also because there is already a
> > > > processor
> > > > > > > that will split a file into multiple flowfiles. But I wanted to
> > hear
> > > > your
> > > > > > > opinion of what is the best way to go. Do you have a
> > recommendation
> > > > for
> > > > > > me?
> > > > > > > (Maybe the answer is to do both?!)
> > > > > > >
> > > > > > > Thanks for sharing your thoughts.
> > > > > > >
> > > > > > > Uwe
> > > > > > >
> > > > > >
> > > > >
> > > >
> > >
> >
>

Re: Re: Re: Processor: User friendly vs system friendly design

Posted by Adam Taft <ad...@adamtaft.com>.
Uwe,

Great progress.

Do you intend that your processors are part of the Apache NiFi standard
distribution?  If so, I think there is enough here to warrant switching the
discussion over to JIRA tickets with accompanying Github activity.

Since you've split the functionality into two processors, I'd probably
recommend creating two feature enhancement JIRA tickets, one for the CSV
related processor and another for the Velocity processor.  If you agree,
please open JIRA tickets at [1].

Here are some thoughts on your processors:

:: CSV Processor Comments ::

-  For the CSV processor, it's doing a very naive String#split and doesn't
deal with quoted CSV files.  What are your thoughts of providing more
robust CSV handling support?  Perhaps using the opencsv project? [2]

-  I think I like the ability for the CSV processor to deal with multiple
rows.  I know I recommend you try using SplitText for this.  But it
probably wouldn't hurt for this processor to accommodate multi-line CSV
input files.  This way, you can throw a single line file or multi-line
file, it will handle both.  For multiple lines, it would export a new
flowfile for each row in the CSV input.

-  If we support multi-line CSV input files, I think it makes sense that it
could also deal with a header line as well.  And optionally, it could
provide prefix values extracted from the header line.  i.e. if the header
line names columns "name", "email", "dob", the processor would export
flowfile attributes with prefix.name, prefix.email, and prefix.dob (instead
of using numbers).

-  I like using DecimalFormat in your solution.  Maybe we should reference
the DecimalFormat API in your documentation.

-  We should probably consider a name with the phrase "CSV" in it, which I
think is appropriate for the NIFI naming style.  Something like
ConvertCSVToAttributes.  I think the NIFI community will probably help
shape the best name.

:: Velocity Processor Comments ::

-  Is there any harm sending the entire Map<String, String> of flowfile
attributes to Velocity?  Why did you decide to include a regex to filter
the attributes send to the engine?  I don't disagree with this feature, but
wondering if it's required or just a nicety.

-  There are likely some code review and naming items to talk about.  For
example, the reference to the VelocityEngine or Template may not be safely
published.  And the name of the processor should probably reference
Velocity.

Thanks,

Adam

[1] https://issues.apache.org/jira/browse/NIFI
[1] http://opencsv.sourceforge.net



On Sat, Mar 19, 2016 at 2:09 PM, Uwe Geercken <uw...@web.de> wrote:

> Adam,
>
> ok. I got it. This way one can use any flowfile attributes and merge them
> with the template.
>
> I have worked all day on the two processors and it does work - very cool.
> Will have to do more error testing though.
>
> So the SplitToAttribute processor does the splitting of the flowfile
> content. It has three properties:
>
> Attribute prefix: how the attributes will be prefixed
> Field seperator: used to split the content
> Field Number format: how to format the positional number of the field
>
> The MergeTemplate processor merges attributes from a flowfile with the
> Apache Velocity template. It has three properties:
>
> Template path: where the template is stored
> Template name: name of the template
> Attribute filter: which fields to make available to the template engine
>
> And then I have two relationships outgoing from the MergeTemplate
> processor: One for the original content of the flowfile and one for the
> result of the merge of the attributes with the template.
>
> Will do more testing. And still my additionalDetails.html documentation
> does not work.
>
> Rgds,
>
> Uwe
>
>
> > Gesendet: Freitag, 18. März 2016 um 22:41 Uhr
> > Von: "Adam Taft" <ad...@adamtaft.com>
> > An: dev@nifi.apache.org
> > Betreff: Re: Re: Processor: User friendly vs system friendly design
> >
> > Uwe,
> >
> > I'll take a look at your code sometime soon.  However, just to point you
> in
> > the direction, I'd suggest extracting your single line CSV data into
> > flowfile attributes named as you've demonstrated.  i.e.  create a
> processor
> > which reads each CSV column as a flowfile attribute, using a configured
> > naming convention.
> >
> > For example, using "column" as your prefix with your example input, you'd
> > end up with a single flowfile with attributes like:
> >
> > column0 = Peterson
> > column1 = Jenny
> > column2 = New York
> > column3 = USA
> >
> > Flowfile attributes are effectively a Map<String,String>.  So in your
> > Velocity processor, you would pass the Map of flowfile attributes to the
> > template engine and record the results to the flowfile content.
> >
> > Using SplitText seems correct up front (though like you said, you lose
> the
> > CSV header line).  You'd need two additional processors, from my
> > perspective:
> >
> > (input) -> SplitText -> ExtractCSVColumns -> ApplyVelocityTemplate ->
> > (output)
> >
> > It's the "​split row into fields and merge with template" that we would
> > want to separate into two processors instead of one.
> >
> > You're very much on the right track, I believe.  If the above doesn't
> help,
> > I'll try and jump in on a code example when I can.
> >
> > Adam
> >
> >
> > On Fri, Mar 18, 2016 at 5:04 PM, Uwe Geercken <uw...@web.de>
> wrote:
> >
> > > Adam,
> > >
> > > I don't see an obvious way for your suggestion of "Read columns from a
> > > single CSV line into flowfile attributes." - I would need your advice
> how I
> > > can achieve it.
> > >
> > > Thinking about it in more detail, I have following issues:
> > > - the incomming flowfile may have many columns. so adding the columns
> > > manually as attributes with UpdateAttributes is not feasible
> > > - I have setup a flow where I use SplitText to divide the flowfile into
> > > multiple flowfiles, so there won't be a header row I can use to get the
> > > column names. So I think I can only use abstract column names plus a
> > > running number. e.g. column0, column1, etc.
> > >
> > > So for the moment I have coded the processor like described below. At
> the
> > > moment I am still "thinking in CSV" but I will check it with other
> formats
> > > later. The user can steer follwoing settings: path where the template
> is
> > > stored, name of the template file, the label for the columns (I call it
> > > prefix) and the separator based on which the split of the row is done.
> > >
> > > Example Flowfile content (user has chosen "comma" as separator:
> > >
> > > Peterson, Jenny, New York, USA
> > >
> > > Example template (user has chosen "column" as the prefix):
> > >
> > > {
> > >         "name": "$column0",
> > >         "first": "$column1",
> > >         "city": "$column2",
> > >         "country": "$column3"
> > > }
> > >
> > > Example flow:
> > >
> > > GetFile: Get CSV File >> SplitText : split into multiple flowfiles, one
> > > per row >> TemplateProcessor:
> > > ​​
> > > split row into fields and merge with template >> MergeContent: merge
> > > flowfiles into one >> PutFile: put the file to the filesystem
> > >
> > > Example result:
> > >
> > > {
> > >         "name": "Peterson",
> > >         "first": "Jenny",
> > >         "city": "New York",
> > >         "country": "USA"
> > >  }
> > >
> > > I will test the processor now for larger files, empty files and other
> > > exceptions. If you are interested the code is here:
> > >
> > > https://github.com/uwegeercken/nifi_processors
> > >
> > > Greetings,
> > >
> > > Uwe
> > >
> > >
> > >
> > > > Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> > > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > > An: dev@nifi.apache.org
> > > > Betreff: Re: Processor: User friendly vs system friendly design
> > > >
> > > > Uwe,
> > > >
> > > > The Developer Guide[1] and Contributor Guide[2] are pretty solid.
> The
> > > > Developer Guide has a section dealing with reading & writing flowfile
> > > > attributes.  Please check these out, and then if you have any
> specific
> > > > questions, please feel free to reply.
> > > >
> > > > For inclusion in NIFI directly, you'd want to create a NIFI Jira
> ticket
> > > > mentioning the new feature, and then fork the NIFI project in Github
> and
> > > > send a Pull Request referencing the ticket.  However, if you just
> want
> > > some
> > > > feedback on suitability and consideration for inclusion, using your
> own
> > > > personal Github project and sending a link would be fine.
> > > >
> > > > Having a template conversion processor would be a nice addition.
> Making
> > > it
> > > > generic to support Velocity, FreeMarker, and others might be really
> nice.
> > > > Extra bonus points for Markdown or Asciidoc transforms as well (but
> these
> > > > might be too separate of a use case).
> > > >
> > > > Hope this helps.
> > > >
> > > > Adam
> > > >
> > > > [1]  http://nifi.apache.org/developer-guide.html
> > > >
> > > > [2]
> https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> > > >
> > > >
> > > >
> > > >
> > > > On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de>
> > > wrote:
> > > >
> > > > > Adam,
> > > > >
> > > > > interesting and I agree. that sounds very good.
> > > > >
> > > > > can you give me short tip of how to access attributes from code?
> > > > >
> > > > > once I have something usable or for testing where would I publish
> it?
> > > just
> > > > > on my github site? or is there a place for sharing?
> > > > >
> > > > > greetings
> > > > >
> > > > > Uwe
> > > > >
> > > > >
> > > > >
> > > > > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > > > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > > > An: dev@nifi.apache.org
> > > > > Betreff: Re: Processor: User friendly vs system friendly design
> > > > > I'm probably on the far end of favoring composibility and processor
> > > reuse.
> > > > > In this case, I would even go one step further and suggest that
> you're
> > > > > talking about three separate operations:
> > > > >
> > > > > 1. Split a multi-line CSV input file into individual single line
> > > flowfiles.
> > > > > 2. Read columns from a single CSV line into flowfile attributes.
> > > > > 3. Pass flowfile attributes into the Velocity transform processor.
> > > > >
> > > > > The point here, have you considered driving your Velocity template
> > > > > transform using flowfile attributes as opposed to CSV? Flowfile
> > > attributes
> > > > > are NIFI's lowest common data representation, many many processors
> > > create
> > > > > attributes which would enable your Velocity processor to be used by
> > > more
> > > > > than just CSV input.
> > > > >
> > > > > Adam
> > > > >
> > > > >
> > > > >
> > > > > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <
> uwe.geercken@web.de>
> > > > > wrote:
> > > > >
> > > > > >
> > > > > > Hello,
> > > > > >
> > > > > > my first mailing here. I am a Java developer, using Apache
> Velocity,
> > > > > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am
> > > very
> > > > > much
> > > > > > a data guy.
> > > > > >
> > > > > > I have used Nifi for a while now and started yesterday of coding
> my
> > > first
> > > > > > processor. I basically do it to widen my knowledge and learn
> > > something
> > > > > new.
> > > > > >
> > > > > > I started with the idea of combining Apache Velocity - a template
> > > engine
> > > > > -
> > > > > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > > > > containing formatting information and some placeholders (and some
> > > limited
> > > > > > logic maybe) and out comes a new set of data, formatted
> differently.
> > > So
> > > > > it
> > > > > > separates the processing logic from the formatting. One could
> create
> > > > > HTML,
> > > > > > XML, Json or other text based formats from it. Easy to use and
> very
> > > > > > efficient.
> > > > > >
> > > > > > Now my question is: Should I rather implement the logic this way
> > > that I
> > > > > > process a whole CSV file - which usually has multiple lines? That
> > > would
> > > > > be
> > > > > > good for the user as he or she has to deal with only one
> processor
> > > doing
> > > > > > the work. But the logic would be more specialized.
> > > > > >
> > > > > > The other way around, I could code the processor to handle one
> row
> > > of the
> > > > > > CSV file and the user will have to come up with a flow that
> divides
> > > the
> > > > > CSV
> > > > > > file into multiple flowfiles before my processor can be used.
> That
> > > is not
> > > > > > so specialized but it requires more preparation work from the
> user.
> > > > > >
> > > > > > I tend to go the second way. Also because there is already a
> > > processor
> > > > > > that will split a file into multiple flowfiles. But I wanted to
> hear
> > > your
> > > > > > opinion of what is the best way to go. Do you have a
> recommendation
> > > for
> > > > > me?
> > > > > > (Maybe the answer is to do both?!)
> > > > > >
> > > > > > Thanks for sharing your thoughts.
> > > > > >
> > > > > > Uwe
> > > > > >
> > > > >
> > > >
> > >
> >
>

Aw: Re: Re: Processor: User friendly vs system friendly design

Posted by Uwe Geercken <uw...@web.de>.
Adam,

ok. I got it. This way one can use any flowfile attributes and merge them with the template.

I have worked all day on the two processors and it does work - very cool. Will have to do more error testing though.

So the SplitToAttribute processor does the splitting of the flowfile content. It has three properties:

Attribute prefix: how the attributes will be prefixed
Field seperator: used to split the content
Field Number format: how to format the positional number of the field

The MergeTemplate processor merges attributes from a flowfile with the Apache Velocity template. It has three properties:

Template path: where the template is stored
Template name: name of the template
Attribute filter: which fields to make available to the template engine

And then I have two relationships outgoing from the MergeTemplate processor: One for the original content of the flowfile and one for the result of the merge of the attributes with the template.

Will do more testing. And still my additionalDetails.html documentation does not work.

Rgds,

Uwe


> Gesendet: Freitag, 18. März 2016 um 22:41 Uhr
> Von: "Adam Taft" <ad...@adamtaft.com>
> An: dev@nifi.apache.org
> Betreff: Re: Re: Processor: User friendly vs system friendly design
>
> Uwe,
> 
> I'll take a look at your code sometime soon.  However, just to point you in
> the direction, I'd suggest extracting your single line CSV data into
> flowfile attributes named as you've demonstrated.  i.e.  create a processor
> which reads each CSV column as a flowfile attribute, using a configured
> naming convention.
> 
> For example, using "column" as your prefix with your example input, you'd
> end up with a single flowfile with attributes like:
> 
> column0 = Peterson
> column1 = Jenny
> column2 = New York
> column3 = USA
> 
> Flowfile attributes are effectively a Map<String,String>.  So in your
> Velocity processor, you would pass the Map of flowfile attributes to the
> template engine and record the results to the flowfile content.
> 
> Using SplitText seems correct up front (though like you said, you lose the
> CSV header line).  You'd need two additional processors, from my
> perspective:
> 
> (input) -> SplitText -> ExtractCSVColumns -> ApplyVelocityTemplate ->
> (output)
> 
> It's the "​split row into fields and merge with template" that we would
> want to separate into two processors instead of one.
> 
> You're very much on the right track, I believe.  If the above doesn't help,
> I'll try and jump in on a code example when I can.
> 
> Adam
> 
> 
> On Fri, Mar 18, 2016 at 5:04 PM, Uwe Geercken <uw...@web.de> wrote:
> 
> > Adam,
> >
> > I don't see an obvious way for your suggestion of "Read columns from a
> > single CSV line into flowfile attributes." - I would need your advice how I
> > can achieve it.
> >
> > Thinking about it in more detail, I have following issues:
> > - the incomming flowfile may have many columns. so adding the columns
> > manually as attributes with UpdateAttributes is not feasible
> > - I have setup a flow where I use SplitText to divide the flowfile into
> > multiple flowfiles, so there won't be a header row I can use to get the
> > column names. So I think I can only use abstract column names plus a
> > running number. e.g. column0, column1, etc.
> >
> > So for the moment I have coded the processor like described below. At the
> > moment I am still "thinking in CSV" but I will check it with other formats
> > later. The user can steer follwoing settings: path where the template is
> > stored, name of the template file, the label for the columns (I call it
> > prefix) and the separator based on which the split of the row is done.
> >
> > Example Flowfile content (user has chosen "comma" as separator:
> >
> > Peterson, Jenny, New York, USA
> >
> > Example template (user has chosen "column" as the prefix):
> >
> > {
> >         "name": "$column0",
> >         "first": "$column1",
> >         "city": "$column2",
> >         "country": "$column3"
> > }
> >
> > Example flow:
> >
> > GetFile: Get CSV File >> SplitText : split into multiple flowfiles, one
> > per row >> TemplateProcessor:
> > ​​
> > split row into fields and merge with template >> MergeContent: merge
> > flowfiles into one >> PutFile: put the file to the filesystem
> >
> > Example result:
> >
> > {
> >         "name": "Peterson",
> >         "first": "Jenny",
> >         "city": "New York",
> >         "country": "USA"
> >  }
> >
> > I will test the processor now for larger files, empty files and other
> > exceptions. If you are interested the code is here:
> >
> > https://github.com/uwegeercken/nifi_processors
> >
> > Greetings,
> >
> > Uwe
> >
> >
> >
> > > Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > An: dev@nifi.apache.org
> > > Betreff: Re: Processor: User friendly vs system friendly design
> > >
> > > Uwe,
> > >
> > > The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
> > > Developer Guide has a section dealing with reading & writing flowfile
> > > attributes.  Please check these out, and then if you have any specific
> > > questions, please feel free to reply.
> > >
> > > For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
> > > mentioning the new feature, and then fork the NIFI project in Github and
> > > send a Pull Request referencing the ticket.  However, if you just want
> > some
> > > feedback on suitability and consideration for inclusion, using your own
> > > personal Github project and sending a link would be fine.
> > >
> > > Having a template conversion processor would be a nice addition.  Making
> > it
> > > generic to support Velocity, FreeMarker, and others might be really nice.
> > > Extra bonus points for Markdown or Asciidoc transforms as well (but these
> > > might be too separate of a use case).
> > >
> > > Hope this helps.
> > >
> > > Adam
> > >
> > > [1]  http://nifi.apache.org/developer-guide.html
> > >
> > > [2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> > >
> > >
> > >
> > >
> > > On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de>
> > wrote:
> > >
> > > > Adam,
> > > >
> > > > interesting and I agree. that sounds very good.
> > > >
> > > > can you give me short tip of how to access attributes from code?
> > > >
> > > > once I have something usable or for testing where would I publish it?
> > just
> > > > on my github site? or is there a place for sharing?
> > > >
> > > > greetings
> > > >
> > > > Uwe
> > > >
> > > >
> > > >
> > > > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > > An: dev@nifi.apache.org
> > > > Betreff: Re: Processor: User friendly vs system friendly design
> > > > I'm probably on the far end of favoring composibility and processor
> > reuse.
> > > > In this case, I would even go one step further and suggest that you're
> > > > talking about three separate operations:
> > > >
> > > > 1. Split a multi-line CSV input file into individual single line
> > flowfiles.
> > > > 2. Read columns from a single CSV line into flowfile attributes.
> > > > 3. Pass flowfile attributes into the Velocity transform processor.
> > > >
> > > > The point here, have you considered driving your Velocity template
> > > > transform using flowfile attributes as opposed to CSV? Flowfile
> > attributes
> > > > are NIFI's lowest common data representation, many many processors
> > create
> > > > attributes which would enable your Velocity processor to be used by
> > more
> > > > than just CSV input.
> > > >
> > > > Adam
> > > >
> > > >
> > > >
> > > > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
> > > > wrote:
> > > >
> > > > >
> > > > > Hello,
> > > > >
> > > > > my first mailing here. I am a Java developer, using Apache Velocity,
> > > > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am
> > very
> > > > much
> > > > > a data guy.
> > > > >
> > > > > I have used Nifi for a while now and started yesterday of coding my
> > first
> > > > > processor. I basically do it to widen my knowledge and learn
> > something
> > > > new.
> > > > >
> > > > > I started with the idea of combining Apache Velocity - a template
> > engine
> > > > -
> > > > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > > > containing formatting information and some placeholders (and some
> > limited
> > > > > logic maybe) and out comes a new set of data, formatted differently.
> > So
> > > > it
> > > > > separates the processing logic from the formatting. One could create
> > > > HTML,
> > > > > XML, Json or other text based formats from it. Easy to use and very
> > > > > efficient.
> > > > >
> > > > > Now my question is: Should I rather implement the logic this way
> > that I
> > > > > process a whole CSV file - which usually has multiple lines? That
> > would
> > > > be
> > > > > good for the user as he or she has to deal with only one processor
> > doing
> > > > > the work. But the logic would be more specialized.
> > > > >
> > > > > The other way around, I could code the processor to handle one row
> > of the
> > > > > CSV file and the user will have to come up with a flow that divides
> > the
> > > > CSV
> > > > > file into multiple flowfiles before my processor can be used. That
> > is not
> > > > > so specialized but it requires more preparation work from the user.
> > > > >
> > > > > I tend to go the second way. Also because there is already a
> > processor
> > > > > that will split a file into multiple flowfiles. But I wanted to hear
> > your
> > > > > opinion of what is the best way to go. Do you have a recommendation
> > for
> > > > me?
> > > > > (Maybe the answer is to do both?!)
> > > > >
> > > > > Thanks for sharing your thoughts.
> > > > >
> > > > > Uwe
> > > > >
> > > >
> > >
> >
>

Re: Re: Processor: User friendly vs system friendly design

Posted by Adam Taft <ad...@adamtaft.com>.
Uwe,

I'll take a look at your code sometime soon.  However, just to point you in
the direction, I'd suggest extracting your single line CSV data into
flowfile attributes named as you've demonstrated.  i.e.  create a processor
which reads each CSV column as a flowfile attribute, using a configured
naming convention.

For example, using "column" as your prefix with your example input, you'd
end up with a single flowfile with attributes like:

column0 = Peterson
column1 = Jenny
column2 = New York
column3 = USA

Flowfile attributes are effectively a Map<String,String>.  So in your
Velocity processor, you would pass the Map of flowfile attributes to the
template engine and record the results to the flowfile content.

Using SplitText seems correct up front (though like you said, you lose the
CSV header line).  You'd need two additional processors, from my
perspective:

(input) -> SplitText -> ExtractCSVColumns -> ApplyVelocityTemplate ->
(output)

It's the "​split row into fields and merge with template" that we would
want to separate into two processors instead of one.

You're very much on the right track, I believe.  If the above doesn't help,
I'll try and jump in on a code example when I can.

Adam


On Fri, Mar 18, 2016 at 5:04 PM, Uwe Geercken <uw...@web.de> wrote:

> Adam,
>
> I don't see an obvious way for your suggestion of "Read columns from a
> single CSV line into flowfile attributes." - I would need your advice how I
> can achieve it.
>
> Thinking about it in more detail, I have following issues:
> - the incomming flowfile may have many columns. so adding the columns
> manually as attributes with UpdateAttributes is not feasible
> - I have setup a flow where I use SplitText to divide the flowfile into
> multiple flowfiles, so there won't be a header row I can use to get the
> column names. So I think I can only use abstract column names plus a
> running number. e.g. column0, column1, etc.
>
> So for the moment I have coded the processor like described below. At the
> moment I am still "thinking in CSV" but I will check it with other formats
> later. The user can steer follwoing settings: path where the template is
> stored, name of the template file, the label for the columns (I call it
> prefix) and the separator based on which the split of the row is done.
>
> Example Flowfile content (user has chosen "comma" as separator:
>
> Peterson, Jenny, New York, USA
>
> Example template (user has chosen "column" as the prefix):
>
> {
>         "name": "$column0",
>         "first": "$column1",
>         "city": "$column2",
>         "country": "$column3"
> }
>
> Example flow:
>
> GetFile: Get CSV File >> SplitText : split into multiple flowfiles, one
> per row >> TemplateProcessor:
> ​​
> split row into fields and merge with template >> MergeContent: merge
> flowfiles into one >> PutFile: put the file to the filesystem
>
> Example result:
>
> {
>         "name": "Peterson",
>         "first": "Jenny",
>         "city": "New York",
>         "country": "USA"
>  }
>
> I will test the processor now for larger files, empty files and other
> exceptions. If you are interested the code is here:
>
> https://github.com/uwegeercken/nifi_processors
>
> Greetings,
>
> Uwe
>
>
>
> > Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> > Von: "Adam Taft" <ad...@adamtaft.com>
> > An: dev@nifi.apache.org
> > Betreff: Re: Processor: User friendly vs system friendly design
> >
> > Uwe,
> >
> > The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
> > Developer Guide has a section dealing with reading & writing flowfile
> > attributes.  Please check these out, and then if you have any specific
> > questions, please feel free to reply.
> >
> > For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
> > mentioning the new feature, and then fork the NIFI project in Github and
> > send a Pull Request referencing the ticket.  However, if you just want
> some
> > feedback on suitability and consideration for inclusion, using your own
> > personal Github project and sending a link would be fine.
> >
> > Having a template conversion processor would be a nice addition.  Making
> it
> > generic to support Velocity, FreeMarker, and others might be really nice.
> > Extra bonus points for Markdown or Asciidoc transforms as well (but these
> > might be too separate of a use case).
> >
> > Hope this helps.
> >
> > Adam
> >
> > [1]  http://nifi.apache.org/developer-guide.html
> >
> > [2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> >
> >
> >
> >
> > On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de>
> wrote:
> >
> > > Adam,
> > >
> > > interesting and I agree. that sounds very good.
> > >
> > > can you give me short tip of how to access attributes from code?
> > >
> > > once I have something usable or for testing where would I publish it?
> just
> > > on my github site? or is there a place for sharing?
> > >
> > > greetings
> > >
> > > Uwe
> > >
> > >
> > >
> > > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > > Von: "Adam Taft" <ad...@adamtaft.com>
> > > An: dev@nifi.apache.org
> > > Betreff: Re: Processor: User friendly vs system friendly design
> > > I'm probably on the far end of favoring composibility and processor
> reuse.
> > > In this case, I would even go one step further and suggest that you're
> > > talking about three separate operations:
> > >
> > > 1. Split a multi-line CSV input file into individual single line
> flowfiles.
> > > 2. Read columns from a single CSV line into flowfile attributes.
> > > 3. Pass flowfile attributes into the Velocity transform processor.
> > >
> > > The point here, have you considered driving your Velocity template
> > > transform using flowfile attributes as opposed to CSV? Flowfile
> attributes
> > > are NIFI's lowest common data representation, many many processors
> create
> > > attributes which would enable your Velocity processor to be used by
> more
> > > than just CSV input.
> > >
> > > Adam
> > >
> > >
> > >
> > > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
> > > wrote:
> > >
> > > >
> > > > Hello,
> > > >
> > > > my first mailing here. I am a Java developer, using Apache Velocity,
> > > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am
> very
> > > much
> > > > a data guy.
> > > >
> > > > I have used Nifi for a while now and started yesterday of coding my
> first
> > > > processor. I basically do it to widen my knowledge and learn
> something
> > > new.
> > > >
> > > > I started with the idea of combining Apache Velocity - a template
> engine
> > > -
> > > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > > containing formatting information and some placeholders (and some
> limited
> > > > logic maybe) and out comes a new set of data, formatted differently.
> So
> > > it
> > > > separates the processing logic from the formatting. One could create
> > > HTML,
> > > > XML, Json or other text based formats from it. Easy to use and very
> > > > efficient.
> > > >
> > > > Now my question is: Should I rather implement the logic this way
> that I
> > > > process a whole CSV file - which usually has multiple lines? That
> would
> > > be
> > > > good for the user as he or she has to deal with only one processor
> doing
> > > > the work. But the logic would be more specialized.
> > > >
> > > > The other way around, I could code the processor to handle one row
> of the
> > > > CSV file and the user will have to come up with a flow that divides
> the
> > > CSV
> > > > file into multiple flowfiles before my processor can be used. That
> is not
> > > > so specialized but it requires more preparation work from the user.
> > > >
> > > > I tend to go the second way. Also because there is already a
> processor
> > > > that will split a file into multiple flowfiles. But I wanted to hear
> your
> > > > opinion of what is the best way to go. Do you have a recommendation
> for
> > > me?
> > > > (Maybe the answer is to do both?!)
> > > >
> > > > Thanks for sharing your thoughts.
> > > >
> > > > Uwe
> > > >
> > >
> >
>

Aw: Re: Processor: User friendly vs system friendly design

Posted by Uwe Geercken <uw...@web.de>.
Adam,

I don't see an obvious way for your suggestion of "Read columns from a single CSV line into flowfile attributes." - I would need your advice how I can achieve it.

Thinking about it in more detail, I have following issues:
- the incomming flowfile may have many columns. so adding the columns manually as attributes with UpdateAttributes is not feasible
- I have setup a flow where I use SplitText to divide the flowfile into multiple flowfiles, so there won't be a header row I can use to get the column names. So I think I can only use abstract column names plus a running number. e.g. column0, column1, etc.

So for the moment I have coded the processor like described below. At the moment I am still "thinking in CSV" but I will check it with other formats later. The user can steer follwoing settings: path where the template is stored, name of the template file, the label for the columns (I call it prefix) and the separator based on which the split of the row is done.

Example Flowfile content (user has chosen "comma" as separator: 

Peterson, Jenny, New York, USA

Example template (user has chosen "column" as the prefix):

{
	"name": "$column0",
	"first": "$column1",
  	"city": "$column2",
  	"country": "$column3"
}

Example flow:

GetFile: Get CSV File >> SplitText : split into multiple flowfiles, one per row >> TemplateProcessor: split row into fields and merge with template >> MergeContent: merge flowfiles into one >> PutFile: put the file to the filesystem

Example result:

{
	"name": "Peterson",
	"first": "Jenny",
	"city": "New York",
	"country": "USA"
 }

I will test the processor now for larger files, empty files and other exceptions. If you are interested the code is here:

https://github.com/uwegeercken/nifi_processors

Greetings,

Uwe



> Gesendet: Freitag, 18. März 2016 um 18:58 Uhr
> Von: "Adam Taft" <ad...@adamtaft.com>
> An: dev@nifi.apache.org
> Betreff: Re: Processor: User friendly vs system friendly design
>
> Uwe,
> 
> The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
> Developer Guide has a section dealing with reading & writing flowfile
> attributes.  Please check these out, and then if you have any specific
> questions, please feel free to reply.
> 
> For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
> mentioning the new feature, and then fork the NIFI project in Github and
> send a Pull Request referencing the ticket.  However, if you just want some
> feedback on suitability and consideration for inclusion, using your own
> personal Github project and sending a link would be fine.
> 
> Having a template conversion processor would be a nice addition.  Making it
> generic to support Velocity, FreeMarker, and others might be really nice.
> Extra bonus points for Markdown or Asciidoc transforms as well (but these
> might be too separate of a use case).
> 
> Hope this helps.
> 
> Adam
> 
> [1]  http://nifi.apache.org/developer-guide.html
> 
> [2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide
> 
> 
> 
> 
> On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de> wrote:
> 
> > Adam,
> >
> > interesting and I agree. that sounds very good.
> >
> > can you give me short tip of how to access attributes from code?
> >
> > once I have something usable or for testing where would I publish it? just
> > on my github site? or is there a place for sharing?
> >
> > greetings
> >
> > Uwe
> >
> >
> >
> > Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> > Von: "Adam Taft" <ad...@adamtaft.com>
> > An: dev@nifi.apache.org
> > Betreff: Re: Processor: User friendly vs system friendly design
> > I'm probably on the far end of favoring composibility and processor reuse.
> > In this case, I would even go one step further and suggest that you're
> > talking about three separate operations:
> >
> > 1. Split a multi-line CSV input file into individual single line flowfiles.
> > 2. Read columns from a single CSV line into flowfile attributes.
> > 3. Pass flowfile attributes into the Velocity transform processor.
> >
> > The point here, have you considered driving your Velocity template
> > transform using flowfile attributes as opposed to CSV? Flowfile attributes
> > are NIFI's lowest common data representation, many many processors create
> > attributes which would enable your Velocity processor to be used by more
> > than just CSV input.
> >
> > Adam
> >
> >
> >
> > On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
> > wrote:
> >
> > >
> > > Hello,
> > >
> > > my first mailing here. I am a Java developer, using Apache Velocity,
> > > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very
> > much
> > > a data guy.
> > >
> > > I have used Nifi for a while now and started yesterday of coding my first
> > > processor. I basically do it to widen my knowledge and learn something
> > new.
> > >
> > > I started with the idea of combining Apache Velocity - a template engine
> > -
> > > with Nifi. So in comes a CSV file, it gets merged with a template
> > > containing formatting information and some placeholders (and some limited
> > > logic maybe) and out comes a new set of data, formatted differently. So
> > it
> > > separates the processing logic from the formatting. One could create
> > HTML,
> > > XML, Json or other text based formats from it. Easy to use and very
> > > efficient.
> > >
> > > Now my question is: Should I rather implement the logic this way that I
> > > process a whole CSV file - which usually has multiple lines? That would
> > be
> > > good for the user as he or she has to deal with only one processor doing
> > > the work. But the logic would be more specialized.
> > >
> > > The other way around, I could code the processor to handle one row of the
> > > CSV file and the user will have to come up with a flow that divides the
> > CSV
> > > file into multiple flowfiles before my processor can be used. That is not
> > > so specialized but it requires more preparation work from the user.
> > >
> > > I tend to go the second way. Also because there is already a processor
> > > that will split a file into multiple flowfiles. But I wanted to hear your
> > > opinion of what is the best way to go. Do you have a recommendation for
> > me?
> > > (Maybe the answer is to do both?!)
> > >
> > > Thanks for sharing your thoughts.
> > >
> > > Uwe
> > >
> >
>

Re: Processor: User friendly vs system friendly design

Posted by Adam Taft <ad...@adamtaft.com>.
Uwe,

The Developer Guide[1] and Contributor Guide[2] are pretty solid.  The
Developer Guide has a section dealing with reading & writing flowfile
attributes.  Please check these out, and then if you have any specific
questions, please feel free to reply.

For inclusion in NIFI directly, you'd want to create a NIFI Jira ticket
mentioning the new feature, and then fork the NIFI project in Github and
send a Pull Request referencing the ticket.  However, if you just want some
feedback on suitability and consideration for inclusion, using your own
personal Github project and sending a link would be fine.

Having a template conversion processor would be a nice addition.  Making it
generic to support Velocity, FreeMarker, and others might be really nice.
Extra bonus points for Markdown or Asciidoc transforms as well (but these
might be too separate of a use case).

Hope this helps.

Adam

[1]  http://nifi.apache.org/developer-guide.html

[2]  https://cwiki.apache.org/confluence/display/NIFI/Contributor+Guide




On Fri, Mar 18, 2016 at 1:36 PM, Uwe Geercken <uw...@web.de> wrote:

> Adam,
>
> interesting and I agree. that sounds very good.
>
> can you give me short tip of how to access attributes from code?
>
> once I have something usable or for testing where would I publish it? just
> on my github site? or is there a place for sharing?
>
> greetings
>
> Uwe
>
>
>
> Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
> Von: "Adam Taft" <ad...@adamtaft.com>
> An: dev@nifi.apache.org
> Betreff: Re: Processor: User friendly vs system friendly design
> I'm probably on the far end of favoring composibility and processor reuse.
> In this case, I would even go one step further and suggest that you're
> talking about three separate operations:
>
> 1. Split a multi-line CSV input file into individual single line flowfiles.
> 2. Read columns from a single CSV line into flowfile attributes.
> 3. Pass flowfile attributes into the Velocity transform processor.
>
> The point here, have you considered driving your Velocity template
> transform using flowfile attributes as opposed to CSV? Flowfile attributes
> are NIFI's lowest common data representation, many many processors create
> attributes which would enable your Velocity processor to be used by more
> than just CSV input.
>
> Adam
>
>
>
> On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de>
> wrote:
>
> >
> > Hello,
> >
> > my first mailing here. I am a Java developer, using Apache Velocity,
> > Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very
> much
> > a data guy.
> >
> > I have used Nifi for a while now and started yesterday of coding my first
> > processor. I basically do it to widen my knowledge and learn something
> new.
> >
> > I started with the idea of combining Apache Velocity - a template engine
> -
> > with Nifi. So in comes a CSV file, it gets merged with a template
> > containing formatting information and some placeholders (and some limited
> > logic maybe) and out comes a new set of data, formatted differently. So
> it
> > separates the processing logic from the formatting. One could create
> HTML,
> > XML, Json or other text based formats from it. Easy to use and very
> > efficient.
> >
> > Now my question is: Should I rather implement the logic this way that I
> > process a whole CSV file - which usually has multiple lines? That would
> be
> > good for the user as he or she has to deal with only one processor doing
> > the work. But the logic would be more specialized.
> >
> > The other way around, I could code the processor to handle one row of the
> > CSV file and the user will have to come up with a flow that divides the
> CSV
> > file into multiple flowfiles before my processor can be used. That is not
> > so specialized but it requires more preparation work from the user.
> >
> > I tend to go the second way. Also because there is already a processor
> > that will split a file into multiple flowfiles. But I wanted to hear your
> > opinion of what is the best way to go. Do you have a recommendation for
> me?
> > (Maybe the answer is to do both?!)
> >
> > Thanks for sharing your thoughts.
> >
> > Uwe
> >
>

Re: Processor: User friendly vs system friendly design

Posted by Uwe Geercken <uw...@web.de>.
Adam,

interesting and I agree. that sounds very good.

can you give me short tip of how to access attributes from code?

once I have something usable or for testing where would I publish it? just on my github site? or is there a place for sharing?

greetings

Uwe
 
 

Gesendet: Freitag, 18. März 2016 um 18:03 Uhr
Von: "Adam Taft" <ad...@adamtaft.com>
An: dev@nifi.apache.org
Betreff: Re: Processor: User friendly vs system friendly design
I'm probably on the far end of favoring composibility and processor reuse.
In this case, I would even go one step further and suggest that you're
talking about three separate operations:

1. Split a multi-line CSV input file into individual single line flowfiles.
2. Read columns from a single CSV line into flowfile attributes.
3. Pass flowfile attributes into the Velocity transform processor.

The point here, have you considered driving your Velocity template
transform using flowfile attributes as opposed to CSV? Flowfile attributes
are NIFI's lowest common data representation, many many processors create
attributes which would enable your Velocity processor to be used by more
than just CSV input.

Adam



On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de> wrote:

>
> Hello,
>
> my first mailing here. I am a Java developer, using Apache Velocity,
> Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very much
> a data guy.
>
> I have used Nifi for a while now and started yesterday of coding my first
> processor. I basically do it to widen my knowledge and learn something new.
>
> I started with the idea of combining Apache Velocity - a template engine -
> with Nifi. So in comes a CSV file, it gets merged with a template
> containing formatting information and some placeholders (and some limited
> logic maybe) and out comes a new set of data, formatted differently. So it
> separates the processing logic from the formatting. One could create HTML,
> XML, Json or other text based formats from it. Easy to use and very
> efficient.
>
> Now my question is: Should I rather implement the logic this way that I
> process a whole CSV file - which usually has multiple lines? That would be
> good for the user as he or she has to deal with only one processor doing
> the work. But the logic would be more specialized.
>
> The other way around, I could code the processor to handle one row of the
> CSV file and the user will have to come up with a flow that divides the CSV
> file into multiple flowfiles before my processor can be used. That is not
> so specialized but it requires more preparation work from the user.
>
> I tend to go the second way. Also because there is already a processor
> that will split a file into multiple flowfiles. But I wanted to hear your
> opinion of what is the best way to go. Do you have a recommendation for me?
> (Maybe the answer is to do both?!)
>
> Thanks for sharing your thoughts.
>
> Uwe
>

Re: Processor: User friendly vs system friendly design

Posted by Adam Taft <ad...@adamtaft.com>.
I'm probably on the far end of favoring composibility and processor reuse.
In this case, I would even go one step further and suggest that you're
talking about three separate operations:

1.  Split a multi-line CSV input file into individual single line flowfiles.
2.  Read columns from a single CSV line into flowfile attributes.
3.  Pass flowfile attributes into the Velocity transform processor.

The point here, have you considered driving your Velocity template
transform using flowfile attributes as opposed to CSV?  Flowfile attributes
are NIFI's lowest common data representation, many many processors create
attributes which would enable your Velocity processor to be used by more
than just CSV input.

Adam



On Fri, Mar 18, 2016 at 11:06 AM, Uwe Geercken <uw...@web.de> wrote:

>
> Hello,
>
> my first mailing here. I am a Java developer, using Apache Velocity,
> Drill, Tomcat, Ant, Pentaho ETL, MongoDb, Mysql and more and I am very much
> a data guy.
>
> I have used Nifi for a while now and started yesterday of coding my first
> processor. I basically do it to widen my knowledge and learn something new.
>
> I started with the idea of combining Apache Velocity - a template engine -
> with Nifi. So in comes a CSV file, it gets merged with a template
> containing formatting information and some placeholders (and some limited
> logic maybe) and out comes a new set of data, formatted differently. So it
> separates the processing logic from the formatting. One could create HTML,
> XML, Json or other text based formats from it. Easy to use and very
> efficient.
>
> Now my question is: Should I rather implement the logic this way that I
> process a whole CSV file - which usually has multiple lines? That would be
> good for the user as he or she has to deal with only one processor doing
> the work. But the logic would be more specialized.
>
> The other way around, I could code the processor to handle one row of the
> CSV file and the user will have to come up with a flow that divides the CSV
> file into multiple flowfiles before my processor can be used. That is not
> so specialized but it requires more preparation work from the user.
>
> I tend to go the second way. Also because there is already a processor
> that will split a file into multiple flowfiles. But I wanted to hear your
> opinion of what is the best way to go. Do you have a recommendation for me?
> (Maybe the answer is to do both?!)
>
> Thanks for sharing your thoughts.
>
> Uwe
>