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

Joseph Witt commented on NIFI-3739:
-----------------------------------

Seeing a lot of these when try to read prov json, write as avro into kafka, 
read from kafka as avro, write as json

This is failing on the publishrecord side

2017-04-27 22:19:40,328 WARN [Timer-Driven Process Thread-7] 
o.a.n.c.t.ContinuallyRunProcessorTask
org.apache.nifi.processor.exception.FlowFileHandlingException: 
StandardFlowFileRecord[uuid=a02d2c58-d678-43d4-ad6d-cf63889abf22,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1493345946912-1, container=default, 
section=1], offset=0, length=874200],offset=0,name=445255826618354,size=874200] 
is not the most recent version of this FlowFile within this session 
(StandardProcessSession[id=85866])
        at 
org.apache.nifi.controller.repository.StandardProcessSession.validateRecordState(StandardProcessSession.java:2865)
        at 
org.apache.nifi.controller.repository.StandardProcessSession.transfer(StandardProcessSession.java:1799)
        at 
org.apache.nifi.processors.kafka.pubsub.PublishKafkaRecord_0_10.onTrigger(PublishKafkaRecord_0_10.java:381)
        at 
org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
        at 
org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1118)
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:144)
        at 
org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47)
        at 
org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:132)
        at 
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
        at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
        at 
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
        at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
        at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
        at java.lang.Thread.run(Thread.java:745)
2017-04-27 22:19:40,367 INFO [Timer-Driven Process Thread-5] 
o.a.n.r.p.s.SocketFlowFileServerProtocol 
SocketFlowFileServerProtocol[CommsID=a605e1a0-12ea-40ff-8ee7-062dbdd04c2d] 
Successfully received 
[StandardFlowFileRecord[uuid=3e5015aa-c38e-42ff-86ce-e755d5fa934b,claim=StandardContentClaim
 [resourceClaim=StandardResourceClaim[id=1493345980352-16, container=default, 
section=16], offset=0, 
length=880659],offset=0,name=445289267675488,size=880659]] (860.02 KB) from 
Peer[url=nifi://10.0.0.15:60538] in 15 milliseconds at a rate of 53.54 MB/sec
2017-04-27 22:19:40,379 INFO [Timer-Driven Process Thread-3] 
o.a.n.r.SiteToSiteProvenanceReportingTask 
SiteToSiteProvenanceReportingTask[id=b258fbd3-015b-1000-57d9-9213c834c0d2] 
Successfully sent 1000 Provenance Events to destination in 86 ms; Transaction 
ID = e4a49837-95d7-44fa-bc4c-abe8c7d8937f; First Event ID = 30100
2017-04-27 22:19:40,477 INFO [Timer-Driven Process Thread-5] 
o.a.n.r.p.s.SocketFlowFileServerProtocol 
SocketFlowFileServerProtocol[CommsID=a605e1a0-12ea-40ff-8ee7-062dbdd04c2d] 
Successfully r

> Create Processors for publishing records to and consuming records from Kafka
> ----------------------------------------------------------------------------
>
>                 Key: NIFI-3739
>                 URL: https://issues.apache.org/jira/browse/NIFI-3739
>             Project: Apache NiFi
>          Issue Type: New Feature
>          Components: Extensions
>            Reporter: Mark Payne
>            Assignee: Mark Payne
>             Fix For: 1.2.0
>
>
> With the new record readers & writers that have been added in now, it would 
> be good to allow records to be pushed to and pulled from kafka. Currently, we 
> support demarcated data but sometimes we can't correctly demarcate data in a 
> way that keeps the format valid (json is a good example). We should have 
> processors that use the record readers and writers for this.



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to