Uploaded image for project: 'Apache NiFi'
  1. Apache NiFi
  2. NIFI-3204

delete hdfs processor throws an error stating transfer relationship not specified even when all relationships are present

    Details

    • Type: Bug
    • Status: Resolved
    • Priority: Major
    • Resolution: Fixed
    • Affects Version/s: 1.0.0, 1.1.0
    • Fix Version/s: 1.2.0
    • Component/s: None
    • Labels:
      None

      Description

      Following flow was setup

      get file -> extract text -> delete hdfs

      A bunch of files were written each having one line which was the path to delete. Some of these path's were files, some were directories and some were patterns. Extract text would extract the line and assign to an attribute which delete hdfs would use to populate the path to delete.

      However the processor would run into an error when ever it tried to process the path which was a pattern matching multiple paths.

      2016-12-14 11:32:43,335 ERROR [Timer-Driven Process Thread-7] o.a.nifi.processors.hadoop.DeleteHDFS DeleteHDFS[id=fed0acf6-0158-1000-b7ab-8cc724e4142d] DeleteHDFS[id=fed0acf6-0158-1000-b7ab-8cc724e4142d] failed to process session due to org.apache.nifi.processor.exception
      .FlowFileHandlingException: StandardFlowFileRecord[uuid=af8be94a-e527-4203-bb87-a0115f84e582,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1481656798897-1, container=default, section=1], offset=6518, length=75],offset=0,name=noyg3p7km8.txt,size=75] tr
      ansfer relationship not specified: org.apache.nifi.processor.exception.FlowFileHandlingException: StandardFlowFileRecord[uuid=af8be94a-e527-4203-bb87-a0115f84e582,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1481656798897-1, container=default, sectio
      n=1], offset=6518, length=75],offset=0,name=noyg3p7km8.txt,size=75] transfer relationship not specified
      2016-12-14 11:32:43,335 ERROR [Timer-Driven Process Thread-7] o.a.nifi.processors.hadoop.DeleteHDFS
      org.apache.nifi.processor.exception.FlowFileHandlingException: StandardFlowFileRecord[uuid=af8be94a-e527-4203-bb87-a0115f84e582,claim=StandardContentClaim [resourceClaim=StandardResourceClaim[id=1481656798897-1, container=default, section=1], offset=6518, length=75],offse
      t=0,name=noyg3p7km8.txt,size=75] transfer relationship not specified
              at org.apache.nifi.controller.repository.StandardProcessSession.checkpoint(StandardProcessSession.java:234) ~[nifi-framework-core-1.1.0.jar:1.1.0]
              at org.apache.nifi.controller.repository.StandardProcessSession.commit(StandardProcessSession.java:304) ~[nifi-framework-core-1.1.0.jar:1.1.0]
              at org.apache.nifi.processor.AbstractProcessor.onTrigger(AbstractProcessor.java:28) ~[nifi-api-1.1.0.jar:1.1.0]
              at org.apache.nifi.controller.StandardProcessorNode.onTrigger(StandardProcessorNode.java:1099) ~[nifi-framework-core-1.1.0.jar:1.1.0]
              at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:136) [nifi-framework-core-1.1.0.jar:1.1.0]
              at org.apache.nifi.controller.tasks.ContinuallyRunProcessorTask.call(ContinuallyRunProcessorTask.java:47) [nifi-framework-core-1.1.0.jar:1.1.0]
              at org.apache.nifi.controller.scheduling.TimerDrivenSchedulingAgent$1.run(TimerDrivenSchedulingAgent.java:132) [nifi-framework-core-1.1.0.jar:1.1.0]
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511) [na:1.8.0_92]
              at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308) [na:1.8.0_92]
              at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180) [na:1.8.0_92]
              at java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294) [na:1.8.0_92]
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142) [na:1.8.0_92]
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617) [na:1.8.0_92]
              at java.lang.Thread.run(Thread.java:745) [na:1.8.0_92]
      2016-12-14 11:32:43,335 WARN [Timer-Driven Process Thread-7] o.a.nifi.processors.hadoop.DeleteHDFS DeleteHDFS[id=fed0acf6-0158-1000-b7ab-8cc724e4142d] Processor Administratively Yielded for 1 sec due to processing failure
      

        Issue Links

          Activity

          Hide
          francoisprunier François Prunier added a comment -

          This only happens if the processor handles an incoming flowfile, ie !context.hasIncomingConnection() is true.

          I have a patch for it but it breaks the semantics of the processor: what do we do with the original flowfile ?

          Show
          francoisprunier François Prunier added a comment - This only happens if the processor handles an incoming flowfile, ie !context.hasIncomingConnection() is true. I have a patch for it but it breaks the semantics of the processor: what do we do with the original flowfile ?
          Hide
          pvillard Pierre Villard added a comment -

          Hi François Prunier, could you submit a PR and we will be able to start from here?

          Show
          pvillard Pierre Villard added a comment - Hi François Prunier , could you submit a PR and we will be able to start from here?
          Hide
          githubbot ASF GitHub Bot added a comment -

          GitHub user francoisprunier opened a pull request:

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

          NIFI-3204 fix handling deleting a path with a wildcard when the processor is in…

          …ssor is invoqued via an incoming flowfile

          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:

              1. 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.
            Yes
          • [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?
              1. 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?
          • [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties?
              1. For documentation related changes:
          • [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
              1. 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/francoisprunier/nifi nifi-3204

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

          https://github.com/apache/nifi/pull/1561.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 #1561


          commit aeab1feeb3c5f47858793e00425b26cc9f2d5b0b
          Author: Francois Prunier <francois.prunier@hurence.com>
          Date: 2017-03-03T10:41:42Z

          NIFI-3204 fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile


          Show
          githubbot ASF GitHub Bot added a comment - GitHub user francoisprunier opened a pull request: https://github.com/apache/nifi/pull/1561 NIFI-3204 fix handling deleting a path with a wildcard when the processor is in… …ssor is invoqued via an incoming flowfile 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. Yes [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? [ ] If adding new Properties, have you added .displayName in addition to .name (programmatic access) for each of the new properties? For documentation related changes: [ ] 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/francoisprunier/nifi nifi-3204 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/nifi/pull/1561.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 #1561 commit aeab1feeb3c5f47858793e00425b26cc9f2d5b0b Author: Francois Prunier <francois.prunier@hurence.com> Date: 2017-03-03T10:41:42Z NIFI-3204 fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile
          Hide
          francoisprunier François Prunier added a comment -

          Hi Pierre Villard,

          Sure, see https://github.com/apache/nifi/pull/1561. The problem I'm seeing at the moment, is that we have a different behaviour whether or not the processor has an incoming flowfile.

          No incoming flowfile -> one flow file emitted per path deleted
          Incoming flowfile -> only one flowfile emitted regardless of the number of path deleted.

          I handled it that way because it makes more sense to me, but it's not satisfying IMO. I think we need a property to control the behaviour of the output of the processor, to control if we have one flowfile per path deleted or one flowfile per incoming flowfile.

          Show
          francoisprunier François Prunier added a comment - Hi Pierre Villard , Sure, see https://github.com/apache/nifi/pull/1561 . The problem I'm seeing at the moment, is that we have a different behaviour whether or not the processor has an incoming flowfile. No incoming flowfile -> one flow file emitted per path deleted Incoming flowfile -> only one flowfile emitted regardless of the number of path deleted. I handled it that way because it makes more sense to me, but it's not satisfying IMO. I think we need a property to control the behaviour of the output of the processor, to control if we have one flowfile per path deleted or one flowfile per incoming flowfile.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user joewitt commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1561#discussion_r104317162

          — Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java —
          @@ -141,20 +141,33 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro
          }

          Map<String, String> attributes = Maps.newHashMapWithExpectedSize(2);
          + boolean foundMissingFile = false;
          for (Path path : pathList) {
          attributes.put("filename", path.getName());
          attributes.put("path", path.getParent().toString());
          if (fileSystem.exists(path)) {
          fileSystem.delete(path, context.getProperty(RECURSIVE).asBoolean());
          +
          if (!context.hasIncomingConnection())

          { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); }
          • session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS);
            +
            } else
            Unknown macro: { getLogger().warn("File (" + path + ") does not exist"); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_FAILURE); } + + }

            + }
            + if (context.hasIncomingConnection()) {
            + // TODO we only put the last path deleted, change the semantic of the processor?

              • End diff –

          We should remove this 'TODO' and instead document the behavior you're describing on the capability decription/and/or the WritesAttribute section. If it is important for a user to see a listing of deleted items they should not use the wildcard here but rather use ListHDFS to create an explicit set of files and this processor should be honoring precisely the items specified.

          Show
          githubbot ASF GitHub Bot added a comment - Github user joewitt commented on a diff in the pull request: https://github.com/apache/nifi/pull/1561#discussion_r104317162 — Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java — @@ -141,20 +141,33 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro } Map<String, String> attributes = Maps.newHashMapWithExpectedSize(2); + boolean foundMissingFile = false; for (Path path : pathList) { attributes.put("filename", path.getName()); attributes.put("path", path.getParent().toString()); if (fileSystem.exists(path)) { fileSystem.delete(path, context.getProperty(RECURSIVE).asBoolean()); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); } session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); + } else Unknown macro: { getLogger().warn("File (" + path + ") does not exist"); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_FAILURE); } + + } + } + if (context.hasIncomingConnection()) { + // TODO we only put the last path deleted, change the semantic of the processor? End diff – We should remove this 'TODO' and instead document the behavior you're describing on the capability decription/and/or the WritesAttribute section. If it is important for a user to see a listing of deleted items they should not use the wildcard here but rather use ListHDFS to create an explicit set of files and this processor should be honoring precisely the items specified.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user joewitt commented on a diff in the pull request:

          https://github.com/apache/nifi/pull/1561#discussion_r104317380

          — Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java —
          @@ -141,20 +141,33 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro
          }

          Map<String, String> attributes = Maps.newHashMapWithExpectedSize(2);
          + boolean foundMissingFile = false;
          for (Path path : pathList) {
          attributes.put("filename", path.getName());
          attributes.put("path", path.getParent().toString());
          if (fileSystem.exists(path)) {
          fileSystem.delete(path, context.getProperty(RECURSIVE).asBoolean());
          +
          if (!context.hasIncomingConnection())

          { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); }
          • session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS);
            +
            } else
            Unknown macro: { getLogger().warn("File (" + path + ") does not exist"); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_FAILURE); } + + }

            + }
            + if (context.hasIncomingConnection()) {
            + // TODO we only put the last path deleted, change the semantic of the processor?
            + if (!foundMissingFile) {

              • End diff –

          Looks like 'foundMissingFile' is set to false above and then there is this if/else. Perhaps this is incomplete? Goes with the 'todo'?

          Show
          githubbot ASF GitHub Bot added a comment - Github user joewitt commented on a diff in the pull request: https://github.com/apache/nifi/pull/1561#discussion_r104317380 — Diff: nifi-nar-bundles/nifi-hadoop-bundle/nifi-hdfs-processors/src/main/java/org/apache/nifi/processors/hadoop/DeleteHDFS.java — @@ -141,20 +141,33 @@ public void onTrigger(ProcessContext context, ProcessSession session) throws Pro } Map<String, String> attributes = Maps.newHashMapWithExpectedSize(2); + boolean foundMissingFile = false; for (Path path : pathList) { attributes.put("filename", path.getName()); attributes.put("path", path.getParent().toString()); if (fileSystem.exists(path)) { fileSystem.delete(path, context.getProperty(RECURSIVE).asBoolean()); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); } session.transfer(session.putAllAttributes(flowFile, attributes), REL_SUCCESS); + } else Unknown macro: { getLogger().warn("File (" + path + ") does not exist"); + if (!context.hasIncomingConnection()) { flowFile = session.create(); + session.transfer(session.putAllAttributes(flowFile, attributes), REL_FAILURE); } + + } + } + if (context.hasIncomingConnection()) { + // TODO we only put the last path deleted, change the semantic of the processor? + if (!foundMissingFile) { End diff – Looks like 'foundMissingFile' is set to false above and then there is this if/else. Perhaps this is incomplete? Goes with the 'todo'?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user joewitt commented on the issue:

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

          The more I look at this processor the more I think the mistake was in allowing it to have wildcards and to run without incoming flow files. Had it been restricted to simply deleting the content of a path specified by a given flowfile it would be much more clear.

          Show
          githubbot ASF GitHub Bot added a comment - Github user joewitt commented on the issue: https://github.com/apache/nifi/pull/1561 The more I look at this processor the more I think the mistake was in allowing it to have wildcards and to run without incoming flow files. Had it been restricted to simply deleting the content of a path specified by a given flowfile it would be much more clear.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user joewitt commented on the issue:

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

          @francoisprunier @pvillard31 Hello. I've attached a patch file to the JIRA https://issues.apache.org/jira/browse/NIFI-3204. The documentation of the processor did not declare its behavior one way or another though tests suggested it created flowfiles for things it deleted. Instead, I have made the behavior and logic more explicit and simplified and updated to the documentation and tests to align to that. For more detailed knowledge of what was deleted/etc.. users should take advantage of this processor in combination with ListHDFS.

          The patch file I provided applies overtop this PR as of commit aeab1feeb3c5f47858793e00425b26cc9f2d5b0b. You can apply the patch on top of your PR and advise if this takes care of the problem and you agree with the fix.

          Thanks

          Show
          githubbot ASF GitHub Bot added a comment - Github user joewitt commented on the issue: https://github.com/apache/nifi/pull/1561 @francoisprunier @pvillard31 Hello. I've attached a patch file to the JIRA https://issues.apache.org/jira/browse/NIFI-3204 . The documentation of the processor did not declare its behavior one way or another though tests suggested it created flowfiles for things it deleted. Instead, I have made the behavior and logic more explicit and simplified and updated to the documentation and tests to align to that. For more detailed knowledge of what was deleted/etc.. users should take advantage of this processor in combination with ListHDFS. The patch file I provided applies overtop this PR as of commit aeab1feeb3c5f47858793e00425b26cc9f2d5b0b. You can apply the patch on top of your PR and advise if this takes care of the problem and you agree with the fix. Thanks
          Hide
          francoisprunier François Prunier added a comment -

          Thanks! I'm away from this office at the moment, I'll have a look at your code by the end of the week.

          Show
          francoisprunier François Prunier added a comment - Thanks! I'm away from this office at the moment, I'll have a look at your code by the end of the week.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user francoisprunier commented on the issue:

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

          Hi @joewitt,

          I've successfully tested your patch. The thing I'm concerned about is the change of behaviour when no incoming flow is connected to the processor, which breaks the contract existing users of the processor may rely on.

          Before:

          • incoming flow: creates one flow file per deleted file: BROKEN when more than one file deleted
          • no incoming flow: creates one flow file per deleted file per run: WORKS

          After:

          • incoming flow: "creates" only one flow file per incoming flowfile: WORKS when more than one file deleted
          • no incoming flow: creates only one flow file per run: WORKS but behaviour is changed

          On the one hand I like your approach better, but on the other hand breaking users' flows is not great.

          Show
          githubbot ASF GitHub Bot added a comment - Github user francoisprunier commented on the issue: https://github.com/apache/nifi/pull/1561 Hi @joewitt, I've successfully tested your patch. The thing I'm concerned about is the change of behaviour when no incoming flow is connected to the processor, which breaks the contract existing users of the processor may rely on. Before: incoming flow: creates one flow file per deleted file: BROKEN when more than one file deleted no incoming flow: creates one flow file per deleted file per run: WORKS After: incoming flow: "creates" only one flow file per incoming flowfile: WORKS when more than one file deleted no incoming flow: creates only one flow file per run: WORKS but behaviour is changed On the one hand I like your approach better, but on the other hand breaking users' flows is not great.
          Hide
          francoisprunier François Prunier added a comment -

          Hi Joseph Witt,

          To continue with the above, should I merge it anyway ?

          Show
          francoisprunier François Prunier added a comment - Hi Joseph Witt , To continue with the above, should I merge it anyway ?
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user joewitt commented on the issue:

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

          @francoisprunier sorry for the delay in responding. Will try to come back to this soon!

          Show
          githubbot ASF GitHub Bot added a comment - Github user joewitt commented on the issue: https://github.com/apache/nifi/pull/1561 @francoisprunier sorry for the delay in responding. Will try to come back to this soon!
          Hide
          joewitt Joseph Witt added a comment - - edited

          François Prunier Yeah i just read back through to remember all this. The behavior is changed but it was previously undeclared and not complete/correct. This approach is now more correct, documented, and for users that need the previous behavior this can be accomplished by using List and Delete together. I am a +1 to merge your patch and my patch on top assuming you're good with it still.

          Show
          joewitt Joseph Witt added a comment - - edited François Prunier Yeah i just read back through to remember all this. The behavior is changed but it was previously undeclared and not complete/correct. This approach is now more correct, documented, and for users that need the previous behavior this can be accomplished by using List and Delete together. I am a +1 to merge your patch and my patch on top assuming you're good with it still.
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user francoisprunier commented on the issue:

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

          @joewitt, I've applied your patch and pushed it, we should be good to go.

          Show
          githubbot ASF GitHub Bot added a comment - Github user francoisprunier commented on the issue: https://github.com/apache/nifi/pull/1561 @joewitt, I've applied your patch and pushed it, we should be good to go.
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 61c799d88b53c72cd38fad820284ecd115a6cf1b in nifi's branch refs/heads/master from Francois Prunier
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=61c799d ]

          NIFI-3204 This closes #1561. fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile
          applied Joseph Witts patch from NIFI-3204 JIRA

          Signed-off-by: joewitt <joewitt@apache.org>

          Show
          jira-bot ASF subversion and git services added a comment - Commit 61c799d88b53c72cd38fad820284ecd115a6cf1b in nifi's branch refs/heads/master from Francois Prunier [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=61c799d ] NIFI-3204 This closes #1561. fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile applied Joseph Witts patch from NIFI-3204 JIRA Signed-off-by: joewitt <joewitt@apache.org>
          Hide
          jira-bot ASF subversion and git services added a comment -

          Commit 61c799d88b53c72cd38fad820284ecd115a6cf1b in nifi's branch refs/heads/master from Francois Prunier
          [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=61c799d ]

          NIFI-3204 This closes #1561. fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile
          applied Joseph Witts patch from NIFI-3204 JIRA

          Signed-off-by: joewitt <joewitt@apache.org>

          Show
          jira-bot ASF subversion and git services added a comment - Commit 61c799d88b53c72cd38fad820284ecd115a6cf1b in nifi's branch refs/heads/master from Francois Prunier [ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=61c799d ] NIFI-3204 This closes #1561. fix handling deleting a path with a wildcard when the processor is invoqued via an incoming flowfile applied Joseph Witts patch from NIFI-3204 JIRA Signed-off-by: joewitt <joewitt@apache.org>
          Hide
          githubbot ASF GitHub Bot added a comment -

          Github user asfgit closed the pull request at:

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

          Show
          githubbot ASF GitHub Bot added a comment - Github user asfgit closed the pull request at: https://github.com/apache/nifi/pull/1561
          Hide
          joewitt Joseph Witt added a comment -

          Thanks François Prunier! We're all set +1 merged to master.

          Show
          joewitt Joseph Witt added a comment - Thanks François Prunier ! We're all set +1 merged to master.

            People

            • Assignee:
              joewitt Joseph Witt
              Reporter:
              arpitgupta Arpit Gupta
            • Votes:
              0 Vote for this issue
              Watchers:
              7 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development