Re: New Feature - Hot deployment of new processors

2016-03-22 Thread Joe Witt
Hello

It is certainly possible to support hot deployment for only new things
but this would likely leave a really rough user experience.  I think
you make a great point about it being easier for things not currently
being used.  I suspect though people will be upgrading/redeploying
things that they are using.

One alternative approach would be to support rolling upgrades.  How do
you feel about that idea?  This is essentially one of the big goals
that I believe becomes more achievable once we support a nice registry
model for extensions.  Now, obviously if you have only a single node
then rolling isn't quite so 'rolling' but rather restarting.  In a
cluster thought it should be a graceful process.

Thanks
Joe

On Tue, Mar 22, 2016 at 8:02 AM, N H  wrote:
>
>
>
> Hi, Complex "data flow systems" always need hot deployment. Is it possible
> to add "hot deployment for ONLY new processors" ?! It might be too complex
> (or too easy I do not know!)  to allow "full support of hot deployment" for
> all processors (especially for those that are being used in current
> context), but it might be feasible for new processors (or even processors
> that are not used in the current context.) It really helps in real
> scenarios. thanks.
>
>


Re: CSV/delimited to Parquet conversion via Nifi

2016-03-22 Thread Dmitry Goldenberg
Agreed, but probably not the case with XML to Avro. Perhaps ConvertFormat would 
be for a set of the more straightforward conversions.

> On Mar 22, 2016, at 11:30 PM, Tony Kurc  wrote:
> 
> On the intermediate representation: not necessarily needed, and likely a 
> performance hindrance to do so. Consider converting from a CSV to a flat json 
> object. This can be done by streaming through the values, and likely only 
> needing a single input character in memory at a time.
> 
> On Mar 22, 2016 11:07 PM, "Dmitry Goldenberg"  
> wrote:
>> It seems to me that for starters it's great to have the processors which 
>> convert from various input formats to FlowFile, and from FlowFile to various 
>> output formats.  That covers all the cases and it gives the users a chance 
>> to run some extra processors in between which is often handy, and sometimes 
>> necessary.
>> 
>> ConvertFormat sounds cool but I'd agree that it may grow to be "hairy" with 
>> the number of conversions, each with its own set of configuration options.  
>> From that perspective, might it be easier to deal with 2 * N specific 
>> converters, and keep adding them as needed, rather than try to maintain a 
>> large "Swiss knife"?
>> 
>> Would ConvertFormat really be able to avoid having to use some kind of 
>> intermediary in-memory format as the conversion is going on?  If not, why 
>> not let this intermediary format be FlowFile, and if it is FlowFile, then 
>> why not just roll with the ConvertFrom / ConvertTo processors?  That way, 
>> implementing a direct converter is simply a matter of dropping the two 
>> converters next to each other into your dataflow (plus a few in-between 
>> transformations, if necessary). 
>> 
>> Furthermore, a combination of a ConvertFrom and a subsequent ConvertTo could 
>> be saved as a sub-template for reuse, left as an exercise for the user, 
>> driven by the user's specific use-cases.
>> 
>> I just wrote a Dataflow which converts some input XML to Avro, and I suspect 
>> that making such a converter work through a common ConvertFormat would take 
>> quite a few options.  Between the start and the finish, I ended up with: 
>> SplitXml, EvaluateXPath, UpdateAttributes, AttributesToJSON, 
>> ConvertJSONToAvro, MergeContent (after that I have a SetAvroFileExtension 
>> and WriteToHdfs).  Too many options to expose for the XMl-to-Avro use-case, 
>> IMHO, for the common ConvertFormat, even if perhaps my Dataflow can be 
>> optimized to avoid a step or two.
>> 
>> Regards,
>> - Dmitry
>> 
>> 
>> 
>>> On Tue, Mar 22, 2016 at 10:25 PM, Matt Burgess  wrote:
>>> I am +1 for the ConvertFormat processor, the  user experience is so much 
>>> enhanced by the hands-off conversion. Such a capability might be contingent 
>>> on the "dependent properties" concept (in Jira somewhere).
>>> 
>>> Also this guy could get pretty big in terms of footprint, I'd imagine the 
>>> forthcoming Registry might be a good place for it.
>>> 
>>> In general a format translator would probably make for a great Apache 
>>> project :) Martin Fowler has blogged about some ideas like this (w.r.t. 
>>> abstracting translation logic), Tika has done some of this but AFAIK its 
>>> focus is on extraction not transformation. In any case, we could certainly 
>>> capture the idea in NiFi.
>>> 
>>> Regards,
>>> Matt
>>> 
 On Mar 22, 2016, at 9:52 PM, Edmon Begoli  wrote:
 
 Good point. 
 
 I just think that Parquet and ORC are important targets, just as 
 relational/JDBC stores are. 
 
> On Tuesday, March 22, 2016, Tony Kurc  wrote:
> Interesting question. A couple discussion points: If we start doing a 
> processor for each of these conversions, it may become unwieldy (P(x,2) 
> processors, where x is number of data formats?) I'd say maybe a more 
> general ConvertFormat processor may be appropriate, but then 
> configuration and code complexity may suffer. If there is a canonical 
> internal data form and a bunch (2*x) of convertXtocanonical, and 
> convertcanonicaltoX processors, the flow could get complex and the extra 
> transform could be expensive.
> 
>> On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg"  
>> wrote:
>> Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would 
>> it make sense to add a feature request for a ConvertJsonToParquet 
>> processor and a ConvertCsvToParquet processor?
>> 
>> - Dmitry
>> 
>>> On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess  
>>> wrote:
>>> Edmon,
>>> 
>>> NIFI-1663 [1] was created to add ORC support to NiFi. If you have a 
>>> target dataset that has been created with Parquet format, I think you 
>>> can use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in 
>>> Parquet format into Hive, HDFS, etc. Others in the 

Re: CSV/delimited to Parquet conversion via Nifi

2016-03-22 Thread Tony Kurc
On the intermediate representation: not necessarily needed, and likely a
performance hindrance to do so. Consider converting from a CSV to a flat
json object. This can be done by streaming through the values, and likely
only needing a single input character in memory at a time.
On Mar 22, 2016 11:07 PM, "Dmitry Goldenberg" 
wrote:

> It seems to me that for starters it's great to have the processors which
> convert from various input formats to FlowFile, and from FlowFile to
> various output formats.  That covers all the cases and it gives the users a
> chance to run some extra processors in between which is often handy, and
> sometimes necessary.
>
> ConvertFormat sounds cool but I'd agree that it may grow to be "hairy"
> with the number of conversions, each with its own set of configuration
> options.  From that perspective, might it be easier to deal with 2 * N
> specific converters, and keep adding them as needed, rather than try to
> maintain a large "Swiss knife"?
>
> Would ConvertFormat really be able to avoid having to use some kind of
> intermediary in-memory format as the conversion is going on?  If not, why
> not let this intermediary format be FlowFile, and if it is FlowFile, then
> why not just roll with the ConvertFrom / ConvertTo processors?  That way,
> implementing a direct converter is simply a matter of dropping the two
> converters next to each other into your dataflow (plus a few in-between
> transformations, if necessary).
>
> Furthermore, a combination of a ConvertFrom and a subsequent ConvertTo
> could be saved as a sub-template for reuse, left as an exercise for the
> user, driven by the user's specific use-cases.
>
> I just wrote a Dataflow which converts some input XML to Avro, and I
> suspect that making such a converter work through a common ConvertFormat
> would take quite a few options.  Between the start and the finish, I ended
> up with: SplitXml, EvaluateXPath, UpdateAttributes, AttributesToJSON,
> ConvertJSONToAvro, MergeContent (after that I have a SetAvroFileExtension
> and WriteToHdfs).  Too many options to expose for the XMl-to-Avro use-case,
> IMHO, for the common ConvertFormat, even if perhaps my Dataflow can be
> optimized to avoid a step or two.
>
> Regards,
> - Dmitry
>
>
>
> On Tue, Mar 22, 2016 at 10:25 PM, Matt Burgess 
> wrote:
>
>> I am +1 for the ConvertFormat processor, the  user experience is so much
>> enhanced by the hands-off conversion. Such a capability might be contingent
>> on the "dependent properties" concept (in Jira somewhere).
>>
>> Also this guy could get pretty big in terms of footprint, I'd imagine the
>> forthcoming Registry might be a good place for it.
>>
>> In general a format translator would probably make for a great Apache
>> project :) Martin Fowler has blogged about some ideas like this (w.r.t.
>> abstracting translation logic), Tika has done some of this but AFAIK its
>> focus is on extraction not transformation. In any case, we could certainly
>> capture the idea in NiFi.
>>
>> Regards,
>> Matt
>>
>> On Mar 22, 2016, at 9:52 PM, Edmon Begoli  wrote:
>>
>> Good point.
>>
>> I just think that Parquet and ORC are important targets, just as
>> relational/JDBC stores are.
>>
>> On Tuesday, March 22, 2016, Tony Kurc  wrote:
>>
>>> Interesting question. A couple discussion points: If we start doing a
>>> processor for each of these conversions, it may become unwieldy (P(x,2)
>>> processors, where x is number of data formats?) I'd say maybe a more
>>> general ConvertFormat processor may be appropriate, but then configuration
>>> and code complexity may suffer. If there is a canonical internal data form
>>> and a bunch (2*x) of convertXtocanonical, and convertcanonicaltoX
>>> processors, the flow could get complex and the extra transform could be
>>> expensive.
>>> On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg" 
>>> wrote:
>>>
 Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would
 it make sense to add a feature request for a ConvertJsonToParquet processor
 and a ConvertCsvToParquet processor?

 - Dmitry

 On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess 
 wrote:

> Edmon,
>
> NIFI-1663 [1] was created to add ORC support to NiFi. If you have a
> target dataset that has been created with Parquet format, I think you can
> use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in Parquet
> format into Hive, HDFS, etc. Others in the community know a lot more about
> the StoreInKiteDataset processor than I do.
>
> Regards,
> Matt
>
> [1] https://issues.apache.org/jira/browse/NIFI-1663
>
> On Mon, Mar 21, 2016 at 8:25 PM, Edmon Begoli 
> wrote:
>
>>
>> Is there a way to do straight CSV(PSV) to Parquet or ORC conversion
>> via Nifi, or do I always need to push the 

Re: CSV/delimited to Parquet conversion via Nifi

2016-03-22 Thread Dmitry Goldenberg
It seems to me that for starters it's great to have the processors which
convert from various input formats to FlowFile, and from FlowFile to
various output formats.  That covers all the cases and it gives the users a
chance to run some extra processors in between which is often handy, and
sometimes necessary.

ConvertFormat sounds cool but I'd agree that it may grow to be "hairy" with
the number of conversions, each with its own set of configuration options.
>From that perspective, might it be easier to deal with 2 * N specific
converters, and keep adding them as needed, rather than try to maintain a
large "Swiss knife"?

Would ConvertFormat really be able to avoid having to use some kind of
intermediary in-memory format as the conversion is going on?  If not, why
not let this intermediary format be FlowFile, and if it is FlowFile, then
why not just roll with the ConvertFrom / ConvertTo processors?  That way,
implementing a direct converter is simply a matter of dropping the two
converters next to each other into your dataflow (plus a few in-between
transformations, if necessary).

Furthermore, a combination of a ConvertFrom and a subsequent ConvertTo
could be saved as a sub-template for reuse, left as an exercise for the
user, driven by the user's specific use-cases.

I just wrote a Dataflow which converts some input XML to Avro, and I
suspect that making such a converter work through a common ConvertFormat
would take quite a few options.  Between the start and the finish, I ended
up with: SplitXml, EvaluateXPath, UpdateAttributes, AttributesToJSON,
ConvertJSONToAvro, MergeContent (after that I have a SetAvroFileExtension
and WriteToHdfs).  Too many options to expose for the XMl-to-Avro use-case,
IMHO, for the common ConvertFormat, even if perhaps my Dataflow can be
optimized to avoid a step or two.

Regards,
- Dmitry



On Tue, Mar 22, 2016 at 10:25 PM, Matt Burgess  wrote:

> I am +1 for the ConvertFormat processor, the  user experience is so much
> enhanced by the hands-off conversion. Such a capability might be contingent
> on the "dependent properties" concept (in Jira somewhere).
>
> Also this guy could get pretty big in terms of footprint, I'd imagine the
> forthcoming Registry might be a good place for it.
>
> In general a format translator would probably make for a great Apache
> project :) Martin Fowler has blogged about some ideas like this (w.r.t.
> abstracting translation logic), Tika has done some of this but AFAIK its
> focus is on extraction not transformation. In any case, we could certainly
> capture the idea in NiFi.
>
> Regards,
> Matt
>
> On Mar 22, 2016, at 9:52 PM, Edmon Begoli  wrote:
>
> Good point.
>
> I just think that Parquet and ORC are important targets, just as
> relational/JDBC stores are.
>
> On Tuesday, March 22, 2016, Tony Kurc  wrote:
>
>> Interesting question. A couple discussion points: If we start doing a
>> processor for each of these conversions, it may become unwieldy (P(x,2)
>> processors, where x is number of data formats?) I'd say maybe a more
>> general ConvertFormat processor may be appropriate, but then configuration
>> and code complexity may suffer. If there is a canonical internal data form
>> and a bunch (2*x) of convertXtocanonical, and convertcanonicaltoX
>> processors, the flow could get complex and the extra transform could be
>> expensive.
>> On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg" 
>> wrote:
>>
>>> Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would
>>> it make sense to add a feature request for a ConvertJsonToParquet processor
>>> and a ConvertCsvToParquet processor?
>>>
>>> - Dmitry
>>>
>>> On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess 
>>> wrote:
>>>
 Edmon,

 NIFI-1663 [1] was created to add ORC support to NiFi. If you have a
 target dataset that has been created with Parquet format, I think you can
 use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in Parquet
 format into Hive, HDFS, etc. Others in the community know a lot more about
 the StoreInKiteDataset processor than I do.

 Regards,
 Matt

 [1] https://issues.apache.org/jira/browse/NIFI-1663

 On Mon, Mar 21, 2016 at 8:25 PM, Edmon Begoli 
 wrote:

>
> Is there a way to do straight CSV(PSV) to Parquet or ORC conversion
> via Nifi, or do I always need to push the data through some of the
> "data engines" - Drill, Spark, Hive, etc.?
>
>
>
>

>>>


Re: Create row keys for HBase from Json messages

2016-03-22 Thread Hong Li
Hi Bryan,

Thank you very much for tips.  I tested during the day.  They are working
now.

Hong



*Hong Li*

*Centric Consulting*

*In Balance*
(888) 781-7567 office
(614) 296-7644 mobile
www.centricconsulting.com | @Centric 

On Mon, Mar 21, 2016 at 8:33 PM, Bryan Bende  wrote:

> Hong,
>
> Glad to hear you are getting started with NiFi! What do your property
> names look like on EvaluatJsonPath?
>
> Typically if you wanted to extract the effective timestamp, event id, and
> applicant id from your example json, then you would add properties to
> EvaluateJsonPath like the following:
>
> effectiveTimestamp =  $.effectiveTimestamp
> eventId = $.event.id
> eventApplicantId = $.event.applicant.id
>
> Then in PutHBaseJson if you want the Row Id to be the event id, followed
> by applicant id, followed by timestamp, you could do:
>
> ${eventId}_${eventApplicantId}_${effectiveTimestamp}
>
> The above expression with your sample JSON should give you:
>
> 1e9b91398160471f8b6197ad974e2464_1f4a3862fab54e058305e3c73cc13dd3_
> 2015-12-03T23:17:29.874Z
>
> Now if you wanted to timestamp to be the long representation instead of
> the date string, you could do:
>
> ${eventId}_${eventApplicantId}_${effectiveTimestamp:
> toDate("-MM-dd'T'HH:mm:ss.SSS'Z'"):toNumber()}
>
> Let us know if this helps.
>
> -Bryan
>
>
> On Mon, Mar 21, 2016 at 7:54 PM, Hong Li 
> wrote:
>
>> I'm a new user for Nifi, and just started my first Nifi project, where we
>> need to move Json messages into HBase.  After I read the templates and user
>> guide, I see I still need help to learn how to concatenate the values
>> pulled out from the Json messages to form a unique row key for HBase tables.
>>
>> Given the sample message below, I run into errors where I need to create
>> the unique keys for HBase by concatenating values pulled from the messages.
>>
>> {
>> "effectiveTimestamp": "2015-12-03T23:17:29.874Z",
>> "event": {
>> "@class": "events.policy.PolicyCreated",
>> "id": "1e9b91398160471f8b6197ad974e2464",
>> "ipAddress": "10.8.30.145",
>> "policy": {
>> "additionalListedInsureds": [],
>> "address": {
>> "city": "Skokie",
>> "county": "Cook",
>> "id": "b863190a5bf846858eb372fb5f532fe7",
>> "latitude": 42.0101,
>> "longitude": -87.75354,
>> "state": "IL",
>> "street": "5014 Estes Ave",
>> "zip": "60077-3520"
>> },
>> "applicant": {
>> "age": 36,
>> "birthDate": "1979-01-12",
>> "clientId": "191",
>> "creditReport": {
>> "id": "ca5ec932d33d444b880c9a43a6eb7c50",
>> "reasons": [],
>> "referenceNumber": "15317191300474",
>> "status": "NoHit"
>> },
>> "firstName": "Kathy",
>> "gender": "Female",
>> "id": "1f4a3862fab54e058305e3c73cc13dd3",
>> "lastName": "Bockett",
>> "maritalStatus": "Single",
>> "middleName": "Sue",
>> "ssn": "***"
>> },
>> "channelOfOrigin": "PublicWebsite",
>> ... ...
>>
>> For example, in processor EvaluateJsonPath, I could pull out individual
>> values as shown below:
>>
>> $.effectiveTimestamp
>> $.event.id
>> $.event.applicant.id
>>
>>
>> However, when I tried to create the HBase row key there such as
>>
>> ${allAttributes($.event.id, 
>> $event.applicant.id):join($.effectiveTimestamp:toDate('MMM
>> d HH:mm:ss'):toString()}_${uuid}
>>
>>
>> I could not make it work no matter how I modified or simplified the long
>> string.  I must have misunderstood something here.  I don't know if this
>> question has already been asked and answered.
>>
>> Thank you for your help.
>> Hong
>>
>>
>> *Hong Li*
>>
>> *Centric Consulting*
>>
>> *In Balance*
>> (888) 781-7567 office
>> (614) 296-7644 mobile
>> www.centricconsulting.com | @Centric 
>>
>
>


Re: CSV/delimited to Parquet conversion via Nifi

2016-03-22 Thread Edmon Begoli
Good point.

I just think that Parquet and ORC are important targets, just as
relational/JDBC stores are.

On Tuesday, March 22, 2016, Tony Kurc  wrote:

> Interesting question. A couple discussion points: If we start doing a
> processor for each of these conversions, it may become unwieldy (P(x,2)
> processors, where x is number of data formats?) I'd say maybe a more
> general ConvertFormat processor may be appropriate, but then configuration
> and code complexity may suffer. If there is a canonical internal data form
> and a bunch (2*x) of convertXtocanonical, and convertcanonicaltoX
> processors, the flow could get complex and the extra transform could be
> expensive.
> On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg"  > wrote:
>
>> Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would
>> it make sense to add a feature request for a ConvertJsonToParquet processor
>> and a ConvertCsvToParquet processor?
>>
>> - Dmitry
>>
>> On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess > > wrote:
>>
>>> Edmon,
>>>
>>> NIFI-1663 [1] was created to add ORC support to NiFi. If you have a
>>> target dataset that has been created with Parquet format, I think you can
>>> use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in Parquet
>>> format into Hive, HDFS, etc. Others in the community know a lot more about
>>> the StoreInKiteDataset processor than I do.
>>>
>>> Regards,
>>> Matt
>>>
>>> [1] https://issues.apache.org/jira/browse/NIFI-1663
>>>
>>> On Mon, Mar 21, 2016 at 8:25 PM, Edmon Begoli >> > wrote:
>>>

 Is there a way to do straight CSV(PSV) to Parquet or ORC conversion
 via Nifi, or do I always need to push the data through some of the
 "data engines" - Drill, Spark, Hive, etc.?




>>>
>>


Re: CSV/delimited to Parquet conversion via Nifi

2016-03-22 Thread Tony Kurc
Interesting question. A couple discussion points: If we start doing a
processor for each of these conversions, it may become unwieldy (P(x,2)
processors, where x is number of data formats?) I'd say maybe a more
general ConvertFormat processor may be appropriate, but then configuration
and code complexity may suffer. If there is a canonical internal data form
and a bunch (2*x) of convertXtocanonical, and convertcanonicaltoX
processors, the flow could get complex and the extra transform could be
expensive.
On Mar 21, 2016 9:39 PM, "Dmitry Goldenberg" 
wrote:

> Since NiFi has ConvertJsonToAvro and ConvertCsvToAvro processors, would it
> make sense to add a feature request for a ConvertJsonToParquet processor
> and a ConvertCsvToParquet processor?
>
> - Dmitry
>
> On Mon, Mar 21, 2016 at 9:23 PM, Matt Burgess  wrote:
>
>> Edmon,
>>
>> NIFI-1663 [1] was created to add ORC support to NiFi. If you have a
>> target dataset that has been created with Parquet format, I think you can
>> use ConvertCSVtoAvro then StoreInKiteDataset to get flow files in Parquet
>> format into Hive, HDFS, etc. Others in the community know a lot more about
>> the StoreInKiteDataset processor than I do.
>>
>> Regards,
>> Matt
>>
>> [1] https://issues.apache.org/jira/browse/NIFI-1663
>>
>> On Mon, Mar 21, 2016 at 8:25 PM, Edmon Begoli  wrote:
>>
>>>
>>> Is there a way to do straight CSV(PSV) to Parquet or ORC conversion via
>>> Nifi, or do I always need to push the data through some of the
>>> "data engines" - Drill, Spark, Hive, etc.?
>>>
>>>
>>>
>>>
>>
>


What is the ideal way of handling Provenance repository corruption?

2016-03-22 Thread Andre
Hi there,

Quick question.

I have noticed that in the case of disk goes full and you start getting
errors like that

2016-03-23 11:01:00,810 ERROR [Timer-Driven Process Thread-6]
o.a.n.p.standard.RouteOnAttribute
RouteOnAttribute[id=14b8bd3c-ca04-4687-ab72-a863e9370482] Failed to process
session due to org.apache.nifi.processor.exception.ProcessException:
FlowFile Repository failed to update:
org.apache.nifi.processor.exception.ProcessException: FlowFile Repository
failed to update

2016-03-23 11:00:46,470 WARN [Provenance Repository Rollover Thread-2]
o.a.n.p.PersistentProvenanceRepository Unable to merge
./repositories/provenance_repository/journals/1947048771.journal.4 with
other Journal Files due to java.io.FileNotFoundException: Unable to locate
file ./repositories/provenance_repository/journals/1947048771.journal.4

and so it goes.

While I appreciate the recommended setup talks about separate mount points
for each of the repositories, I wonder:

Is there a way of restoring the system that doesn't involve removing the
contents of each repository?

Cheers


PutKafka Processor Time-out Errors with Guarantee Replicated Delivery on NiFi 0.5.1 and Kafka 0.8.2

2016-03-22 Thread indus well
Hello NiFi Experts:

I am getting time-out errors from the PutKafka processor when using
the Guarantee Replicated Delivery option in the Guarantee Delivery property
on NiFi 0.5.1 with Kafka 0.8.2 cluster. However, everything is working as
normal when I switched to the Best Effort option. In addition, NiFi 0.4.1
works just fine with either option.  Here's the error:

19:33:02 CDTERROR478f6a4f-4b79-4253-afe4-ecda8174df67
PutKafka[id=478f6a4f-4b79-4253-afe4-ecda8174df67] Failed to send
StandardFlowFileRecord[uuid=ebcd-5220-45c0-a713-f3d68fac633f,claim=StandardContentClaim
[resourceClaim=StandardResourceClaim[id=1458692540256-122,
container=default, section=122], offset=110349,
length=122],offset=0,name=1033351446389479,size=122] to Kafka; routing to
'failure'; last failure reason reported was
org.apache.kafka.common.errors.TimeoutException: Batch Expired;:
org.apache.kafka.common.errors.TimeoutException: Batch Expired

Is this issue related to Joe's comment in the following thread:

http://mail-archives.apache.org/mod_mbox/nifi-users/201603.mbox/%3CCALJK9a4h%3D_A-LZPsAvWaAKcGu3qR78fWh2572gDh53Z9LeuLFQ%40mail.gmail.com%3E

Please advise.

Thanks,

Indus


nifi.content.repository.archive.max.retention.period

2016-03-22 Thread Andre
Hi there,

I have a testing instance of nifi 0.4.2 running an I've noticed a very
strange behaviour around content archives

When I look I my settings I see:

# Content Repository
nifi.content.repository.implementation=org.apache.nifi.controller.repository.Fil
eSystemRepository
nifi.content.claim.max.appendable.size=10 MB
nifi.content.claim.max.flow.files=100
nifi.content.repository.directory.default=./repositories/content_repository
nifi.content.repository.archive.max.retention.period=12 hours
nifi.content.repository.archive.max.usage.percentage=50%
nifi.content.repository.archive.enabled=true
nifi.content.repository.always.sync=false
nifi.content.viewer.url=/nifi-content-viewer/


Yet, when looking at the archives I can see data that is days older than
the retention period (as an example, some of the data I have is quite old
(we didn't see much data crossing that instance so the issue was never
detected) but we had some changes last week and that became more evident.

Am I correct to assume NiFi should have cleaned archives older than 12
hours?

Cheers


Re: Dataflow architecture for multiple sources

2016-03-22 Thread Andrew Grande
Aurélien,

The choice of a multiplexing channel or multiple dedicated ones is really up to 
any constraints your environment may (not) have. E.g. if you are able to expose 
every port required for a socket-based protocol or no.

On the NiFi side, take a close look at Backpressure here, it will take care of 
data storms:

https://nifi.apache.org/docs/nifi-docs/html/getting-started.html

Andrew

From: Aurélien DEHAY 
> on behalf of 
"aurelien.de...@gmail.com" 
>
Reply-To: "users@nifi.apache.org" 
>
Date: Tuesday, March 22, 2016 at 8:03 AM
To: "users@nifi.apache.org" 
>
Subject: Dataflow architecture for multiple sources


Hello.


I've to make an architecture based on nifi to collect & route data from sources 
to some hadoop/ES cluster.


Sources will have different constraints (from 50msg/s to hundred of thousand, 
not the same latency necessities, not the same protocol, etc.).


I wonder if we should make a processor per data source (e.g. one port for each 
data source), or I can send to a processor per protocol, and route based on 
some attribute afterwards.


I we use a single entry processor per protocol for all data sources, won't 
there be risks on the shared queue, in case of data storm for example?


thanks for any pointer / answer.


Aurélien.


Dataflow architecture for multiple sources

2016-03-22 Thread aurelien.de...@gmail.com
Hello.


I've to make an architecture based on nifi to collect & route data from sources 
to some hadoop/ES cluster.


Sources will have different constraints (from 50msg/s to hundred of thousand, 
not the same latency necessities, not the same protocol, etc.).


I wonder if we should make a processor per data source (e.g. one port for each 
data source), or I can send to a processor per protocol, and route based on 
some attribute afterwards.


I we use a single entry processor per protocol for all data sources, won't 
there be risks on the shared queue, in case of data storm for example?


thanks for any pointer / answer.


Aurélien.


New Feature - Hot deployment of new processors

2016-03-22 Thread N H

 

Hi, Complex "data flow systems" always need hot deployment. Is it possible to 
add "hot deployment for ONLY new processors" ?! It might be too complex (or too 
easy I do not know!)  to allow "full support of hot deployment" for all 
processors (especially for those that are being used in current context), but 
it might be feasible for new processors (or even processors that are not used 
in the current context.) It really helps in real scenarios. thanks.


  

Re: Help on creating that flow that requires processing attributes in a flow content but need to preserve the original flow content

2016-03-22 Thread Conrad Crampton
My 2p. 
If the kaka.key value (very simple json), you could use UpdateAttribute and use 
some expression language - specifically the string manipulation functions to 
extract the part you want.
I like the power or ExecuteProcessor by the way.

And I agree, this community is phenomenally responsive and helpful.

Regards
Conrad




On 21/03/2016, 18:38, "McDermott, Chris Kevin (MSDU - STaTS/StorefrontRemote)" 
 wrote:

>Thanks everyone.  While I’m naturally disappointed that this doesn’t exist, I 
>am hyper charged about the responsiveness and enthusiasm of the NiFi community!
>
>From: Matt Burgess >
>Reply-To: "users@nifi.apache.org" 
>>
>Date: Monday, March 21, 2016 at 1:58 PM
>To: "users@nifi.apache.org" 
>>
>Subject: Re: Help on creating that flow that requires processing attributes in 
>a flow content but need to preserve the original flow content
>
>One way (in NiFi 0.5.0+) is to use the ExecuteScript processor, which gives 
>you full control over the session and flowfile(s).  For example if you had 
>JSON in your "kafka.key" attribute such as "{"data": {"myKey": "myValue"}}" , 
>you could use the following Groovy script to parse out the value of the 
>'data.myKey' field:
>
>def flowfile = session.get()
>if(!flowfile) return
>def json = new 
>groovy.json.JsonSlurper().parseText(flowfile.getAttribute('kafka.key'))
>flowfile = session.putAttribute(flowfile, 'myKey', json.data.myKey)
>session.transfer(flowfile, REL_SUCCESS)
>
>
>I put an example of this up as a Gist 
>(https://gist.github.com/mattyb149/478864017ec70d76f74f)
>
>A possible improvement could be to add a "jsonPath" function to Expression 
>Language, which could take any value (including an attribute) along with a 
>JSONPath expression to evaluate against it...
>
>Regards,
>Matt
>
>On Mon, Mar 21, 2016 at 1:48 PM, McDermott, Chris Kevin (MSDU - 
>STaTS/StorefrontRemote) 
>> wrote:
>Joe,
>
>Thanks for the reply.  I think I was not clear.
>
>The JSON I need to evaluate is in a FlowFile attribute (kafka.key) which I 
>need to be able to evaluate without modifying the original FlowFile content 
>(which was read from the Kafka topic).  What I can’t figure out is how to 
>squirrel away the flowfile content so that I can write the value of the 
>kafka.key attribute to the FlowFile content, so that I can process it with 
>EvaluateJsonPath, and then read content I squirreled away back into the 
>FlowFile content. I considered using the the DistributedMapCache, but there 
>would be no guarantee what I added to the cache would still be there when I 
>needed it back.
>
>
>
>
>On 3/21/16, 1:37 PM, "Joe Witt" 
>> wrote:
>
>>Chris,
>>
>>Sounds like you have the right flow in mind already.  EvaluateJSONPath
>>does not write content.  It merely evaluates the given jsonpath
>>expression against the content of the flowfile and if appropriate
>>creates a flowfile attribute of what it finds.
>>
>>For example if you have JSON from Twitter you can use EvaluateJsonPath
>>and add a property with a name
>>'twitter.user' and a value of '$.user.name'
>>
>>Once you run the tweets through each flow file will have an attribute
>>called 'twitter.user' with the name found in the message.  No
>>manipulation of content at all.  Just promotes things it finds to flow
>>file attributes.
>>
>>Thanks
>>Joe
>>
>>On Mon, Mar 21, 2016 at 1:34 PM, McDermott, Chris Kevin (MSDU -
>>STaTS/StorefrontRemote) 
>>> wrote:
>>> What I need to do is read a file from Kafka.  The Kafka key contains a JSON 
>>> string which I need to turn in FlowFile attributes while preserving the 
>>> original FlowFile content.  Obviously I can use EvaluteJsonPath but that 
>>> necessitates replacing the FlowFile content with the kaka.key attribute, 
>>> thus loosing the original FlowFile content.  I feel like I’m missing 
>>> something fundamental.
>
>
>
> ***This email originated outside SecureData***
>
>Click https://www.mailcontrol.com/sr/MZbqvYs5QwJvpeaetUwhCQ==  to report this 
>email as spam.


SecureData, combating cyber threats
__ 
The information contained in this message or any of its attachments may be 
privileged and confidential and intended for the exclusive use of the intended 
recipient. If you are not the intended recipient any disclosure, reproduction, 
distribution or other dissemination or use of this communications is strictly 
prohibited. The views expressed in this email are those of the individual and 
not necessarily of SecureData Europe Ltd. Any prices quoted are only valid if 
followed up by a formal written