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

ASF subversion and git services commented on NIFI-1764:
-------------------------------------------------------

Commit 2d03489ec5dbc56bcb0dbd7b694cdf030d2532a5 in nifi's branch 
refs/heads/master from [~ozhurakousky]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=2d03489 ]

NIFI-1296, NIFI-1680, NIFI-1764, NIFI-1837, NIFI-1827, NIFI-1699  implemented 
new  Kafka processors that leverage Kafka 0.9 API
- Improved StreamScanner for better performance
- Renamed StreamScanner to StreamDemarcator as suggested by Joe
- Added failure handling logic to ensure both processors can be reset to their 
initial state (as if they were just started)
- Provided comprehensive test suite to validate various aspects of both Publish 
and Consume from Kafka
- Added relevant javadocs
- Added initial additionalDetails docs
- Addressed NPE reported by NIFI-1764
- Life-cycle refactoring for the existing PutKafka to ensure producer restart 
after errors
- Incorporated code changes contributed by Ralph Perko (see NIFI-1837)
- Addressed partition issue in RoundRobinPartitioner discussed in NIFI-1827
- Updated PropertyDescriptor descriptions to reflect their purpose

NIFI-1296 added @Ignore on some Kafka tests to improve test time

NIFI-1296 reworked tests to avoid dependency on embedded Kafka

NIFI-1296 fixed spelling error

NIFI-1296 fixed trailing whitespaces in non-java files

This closes #366


> NullPointerException in PutKafka for failed segments with no delimiter and 
> insufficient producer handling
> ---------------------------------------------------------------------------------------------------------
>
>                 Key: NIFI-1764
>                 URL: https://issues.apache.org/jira/browse/NIFI-1764
>             Project: Apache NiFi
>          Issue Type: Bug
>          Components: Extensions
>    Affects Versions: 1.0.0, 0.7.0
>            Reporter: Christopher McDermott
>            Assignee: Oleg Zhurakousky
>              Labels: patch
>             Fix For: 1.0.0, 0.7.0
>
>         Attachments: 0001-NIFI-1764-numerous-edits-from-reviewing-PR-366.patch
>
>
> This NPE can happen during certain failure cases and it appears to be related 
> to the lack of guarding of the failed segments attribute addition in the case 
> there is no delimiter.  Further, we have observed the PutKafka processor 
> becoming ineffective if the established kafka client starts seeing failed 
> acks/timeouts.  We need to catch those cases and teardown the old client and 
> create a new one instead.
> {code}
> java.lang.NullPointerException: null
> at java.lang.String.<init>(String.java:503) ~[na:1.8.0_45]
> at 
> org.apache.nifi.processors.kafka.PutKafka.buildFailedFlowFileAttributes(PutKafka.java:396)
>  ~[na:na]
> at org.apache.nifi.processors.kafka.PutKafka.onTrigger(PutKafka.java:308) 
> ~[na:na]
> at 
> org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:27)
>  ~[nifi-api-0.7.0-SNAPSHOT.jar:0.7.0-SNAPSHOT]
> at 
> org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1059)
>  ~[nifi-framework-core-0.7.0-SNAPSHOT.jar:0.7.0-SNAPSHOT]
> at 
> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:136)
>  [nifi-framework-core-0.7.0-SNAPSHOT.jar:0.7.0-SNAPSHOT]
> at 
> org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47)
>  [nifi-framework-core-0.7.0-SNAPSHOT.jar:0.7.0-SNAPSHOT]
> at 
> org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:123)
>  [nifi-framework-core-0.7.0-SNAPSHOT.jar:0.7.0-SNAPSHOT]
> at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) 
> [na:1.8.0_45]
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) 
> [na:1.8.0_45]
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
>  [na:1.8.0_45]
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
>  [na:1.8.0_45]
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to