You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by ijokarumawak <gi...@git.apache.org> on 2017/04/07 15:42:56 UTC

[GitHub] nifi pull request #1658: NIFI-3415: Add Rollback on Failure.

GitHub user ijokarumawak opened a pull request:

    https://github.com/apache/nifi/pull/1658

    NIFI-3415: Add Rollback on Failure.

    - Added org.apache.nifi.processor.util.pattern package in nifi-processor-utils containing reusable functions to mix-in 'Rollback on Failure' capability.                                                    - Created a process pattern classes, Put and PutGroup. It will be helpful to standardize Processor implementations.
    - Applied Rollback on Failure to PutSQL, PutHiveQL and PutHiveStreaming.
    - Fixed existing issues on PutHiveStreaming:
      - Output FlowFile Avro format was corrupted by concatenating multiple Avro files.
      - Output FlowFile records had incorrect values because of reusing GenericRecord instance.
    
    Thank you for submitting a contribution to Apache NiFi.
    
    In order to streamline the review of the contribution we ask you
    to ensure the following steps have been taken:
    
    ### For all changes:
    - [x] Is there a JIRA ticket associated with this PR? Is it referenced 
         in the commit message?
    
    - [x] Does your PR title start with NIFI-XXXX where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
    
    - [x] Has your PR been rebased against the latest commit within the target branch (typically master)?
    
    - [x] Is your initial contribution a single, squashed commit?
    
    ### For code changes:
    - [x] Have you ensured that the full suite of tests is executed via mvn -Pcontrib-check clean install at the root nifi folder?
    - [x] Have you written or updated unit tests to verify your changes?
    - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)? 
    - [ ] If applicable, have you updated the LICENSE file, including the main LICENSE file under nifi-assembly?
    - [ ] If applicable, have you updated the NOTICE file, including the main NOTICE file found under nifi-assembly?
    - [x] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
    
    ### For documentation related changes:
    - [x] Have you ensured that format looks appropriate for the output in which it is rendered?
    
    ### Note:
    Please ensure that once the PR is submitted, you check travis-ci for build issues and submit an update to your PR as soon as possible.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/ijokarumawak/nifi nifi-3415

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/1658.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1658
    
----
commit e5c235be647957d8804a3bca004f97b4269cc170
Author: Koji Kawamura <ij...@apache.org>
Date:   2017-03-02T00:51:12Z

    NIFI-3415: Add Rollback on Failure.
    
    - Added org.apache.nifi.processor.util.pattern package in nifi-processor-utils containing reusable functions to mix-in 'Rollback on Failure' capability.                                                    - Created a process pattern classes, Put and PutGroup. It will be helpful to standardize Processor implementations.
    - Applied Rollback on Failure to PutSQL, PutHiveQL and PutHiveStreaming.
    - Fixed existing issues on PutHiveStreaming:
      - Output FlowFile Avro format was corrupted by concatenating multiple Avro files.
      - Output FlowFile records had incorrect values because of reusing GenericRecord instance.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    @mattyb149 As we discussed at #1677, I've added Rollback on Failure to PutDatabaseRecord. I changed many lines of code, but those are basically refactoring on error handling and dividing the existing onTrigger method into several parts for better manageability and clearer execution paths.
    
    I also added context yielding when Rollback on Failure is engaged. Because without doing penalization nor yielding, the failing FlowFile retries too frequently.
    
    Would you review this again? Please let me know your feedback. Thank you!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    With PutHiveQL, I sent one valid statement, then one bad one, then another valid one. The first one was processed correctly and successfully, the second one caused and error and remained in the queue, but the third one was not processed nor was it in the incoming or outgoing (failure/retry) queue. Provenance shows a SEND event for the first one, DROP events for the first and third, and only the CREATE event for the second (aka "bad") one. I was thinking the incoming queue would possibly have the second one (if the error is a ProcessException vs something that routes the flow file) and the third one, or failure/retry (if a failed flow file is routed) would have the second one.  Here's a stack trace showing the processing of the bad record (in case it helps):
    
    ```
    org.apache.nifi.processor.exception.ProcessException: Failed to process org.apache.nifi.processor.util.pattern.ExceptionHandler$$Lambda$397/1081958194@17605416 due to org.apache.hive.service.cli.HiveSQLException: Error while compiling statement: FAILED: ParseException line 1:0 cannot recognize input near 'INERT' 'INTO' 'TABLE'
    	at org.apache.nifi.processor.util.pattern.ExceptionHandler.lambda$createOnGroupError$12(ExceptionHandler.java:210)
    	at org.apache.nifi.processor.util.pattern.ExceptionHandler.lambda$createOnError$11(ExceptionHandler.java:178)
    	at org.apache.nifi.processor.util.pattern.ExceptionHandler$OnError.lambda$andThen$9(ExceptionHandler.java:53)
    	at org.apache.nifi.processor.util.pattern.ExceptionHandler$OnError.lambda$andThen$9(ExceptionHandler.java:53)
    	at org.apache.nifi.processor.util.pattern.ExceptionHandler.execute(ExceptionHandler.java:147)
    	at org.apache.nifi.processors.hive.PutHiveQL.lambda$new$44(PutHiveQL.java:199)
    	at org.apache.nifi.processor.util.pattern.Put.putFlowFiles(Put.java:58)
    	at org.apache.nifi.processor.util.pattern.Put.onTrigger(Put.java:100)
    	at org.apache.nifi.processors.hive.PutHiveQL.lambda$onTrigger$46(PutHiveQL.java:255)
    	at org.apache.nifi.processor.util.pattern.PartialFunctions.onTrigger(PartialFunctions.java:114)
    	at org.apache.nifi.processor.util.pattern.RollbackOnFailure.onTrigger(RollbackOnFailure.java:184)
    	at org.apache.nifi.processors.hive.PutHiveQL.onTrigger(PutHiveQL.java:255)
    	at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1115)
    	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)
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    For dear reviewer(s), I've put template files and description on how it works in this [Gist](https://gist.github.com/ijokarumawak/e552885a29c53ce60012cacef62a3342), please take a look on it, too. Thanks a lot in advance!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    @mattyb149 I found what was wrong. I mistakenly used `remove` method to clear the fetched-but-not-processed FlowFiles from session. I should have routed those to self relationship. Sorry about that and thank you so much for finding this. Fix that and confirmed by unit testing.
    
    I will continue integration test with my Hive environment but just pushed updates so that you can review it at the same time. Please let me know if you find anything. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    +1 LGTM, ran unit tests and tried all 4 processors with Rollback on Failure, everything exhibited the expected behavior. Great addition, thanks much!! Merging to master


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    I tried the following use case:
    
    - 3 FlowFiles containing SQL, the first two INSERT the same value (meaning the second will fail), then third INSERTing a new value
    - PutSQL with Supports Fragmented Transactions set to `true` (with no fragment.* attributes set) and Batch Size 1
    
    I was pleasantly surprised to see that even the first successful statement was rolled back when the second one failed. This was because all 3 flow files were already in the queue, and using Supports Fragmented Transactions without the fragment attributes set will cause the `TransactionalFlowFileFilter` to grab all the flow files (even though the Batch Size is 1). That is existing behavior (although not documented). We can't count on that though, because we don't know how many files will be in the queue when `pollFlowFiles()` is called.
    
    However, when I set Supports Fragmented Transactions to `false` with a Batch Size of 1, then the first and third flow files (the valid ones) were both processed successfully, and the second flow file was retained in the queue. I would've expected after the second file failed, the third one would not be processed. What are your thoughts? Did I configure it incorrectly?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    @mattyb149 I finished testing PutHiveQL and PutHiveStreaming using Hive environment and now it works as expected as shown in the [diagrams in Gist](https://gist.github.com/ijokarumawak/e552885a29c53ce60012cacef62a3342). I squashed commits, and it's ready for the final review cycle. Please take a look. Thank you!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request #1658: NIFI-3415: Add Rollback on Failure.

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/nifi/pull/1658


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    Running with PutHiveStreaming, Rollback on Failure works well when there is one incoming connection, however with two it seemed to accept future flow files (one connection had good data, one bad). I'm not sure we can do anything about this though, perhaps file an Improvement Jira to look into this (or ask @markap14 if it's even possible)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    I've updated this PR to rebase with the latest master and address merge conflicts.
    
    Thanks @mattyb149 for the above two comments.
    
    For the case with multiple incoming relationships, there is a [clear documentation](https://github.com/apache/nifi/blob/master/nifi-api/src/main/java/org/apache/nifi/processor/ProcessSession.java#L172) that behavior is unspecified. I believe we suggest using a funnel to bundle such relationships when ordering among queues is important.
    
    For PutHiveQL, is my understanding correct that your 3rd FlowFile disappeared from incoming queue and it wasn't neither in outgoing queue? I will try to reproduce the scenario and let you know what I find. Thanks for sharing!



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    I've updated the PR and changed how those processors handle failed input FlowFile after it had processed other inputs successfully within the same onTrigger process session. In short, "Put the failed FlowFile back into its input queue then discontinue."
    
    [Gist](https://gist.github.com/ijokarumawak/e552885a29c53ce60012cacef62a3342) is also updated to illustrate what will happen with default behavior and enabled 'Rollback on Failure'.
    
    Any feedback is appreciated. Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi issue #1658: NIFI-3415: Add Rollback on Failure.

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on the issue:

    https://github.com/apache/nifi/pull/1658
  
    @mattyb149 Thanks for catching that! Your expectation was correct, however AbstractProcessor always penalizes FlowFiles being processed when it rollback a process session. The 2nd FlowFile was penalized and the 3rd one was processed before the 2nd.
    
    I updated processors using RollbackOnFailure to not extend AbstractProcessor, so that those can penalize FlowFiles based on whether RollbackOnFailure is enabled or not.
    
    I confirmed that if the 2nd FlowFile failed out of 3 FlowFiles, the 2nd and 3rd FlowFiles stay in the incoming relationship and the 2nd FlowFile will be processed again and again until it gets processed successfully.
    
    Thanks again for your insightful review comments!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---