You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by mattyb149 <gi...@git.apache.org> on 2017/12/18 18:48:23 UTC

[GitHub] nifi pull request #2351: NIFI-4707: Build full component map for ID -> Name ...

GitHub user mattyb149 opened a pull request:

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

    NIFI-4707: Build full component map for ID -> Name association in provenance reporting

    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?
    - [ ] 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:
    - [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/mattyb149/nifi NIFI-4707

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

    https://github.com/apache/nifi/pull/2351.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 #2351
    
----
commit 749d5b6b0251392e54d48d8f7673d2a87f364543
Author: Matthew Burgess <ma...@apache.org>
Date:   2017-12-18T18:44:21Z

    NIFI-4707: Build full component map for ID -> Name association in provenance reporting"

----


---

[GitHub] nifi issue #2351: NIFI-4707: S2SProvenanceReportingTask not returning correc...

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

    https://github.com/apache/nifi/pull/2351
  
    Hi @mattyb149 Thanks for updating this PR. It mostly looks good, however, while I was testing, I found few points those can be improved. I went ahead and added following improvements on top ob your commits. Would you cherry-pick this commit? https://github.com/ijokarumawak/nifi/commit/8effe3b19681ac34594a2f33e9d049ef081730a6
    
    1. "Remote Input/Output Port" port name and process group id can only be retrieved by mapping ConnectionStatus source or destination component id.
    2. When a ProcessGroupId is used to filter events, the filtering should consider PG hierarchy, meaning if PG1 is a child of Root, and PG2 is a child of PG1, and PG1 uuid is used as filter component id, then provenance events happening at PG2 should also be reported.
    
    Other minor improvements:
        - Simplified consumeEvents method signature
        - Refactored ComponentMapHolder methods visibility
        - Renamed componentMap to componentNameMap
        - Throw an exception when the reporting task fails to send provenance data to keep current provenance event index so that events can be consumed again
    
    Thank you!
    



---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r158616813
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/provenance/ProvenanceEventConsumer.java ---
    @@ -218,18 +230,35 @@ private boolean isFilteringEnabled() {
             return componentTypeRegex != null || !eventTypes.isEmpty() || !componentIds.isEmpty();
         }
     
    -    private List<ProvenanceEventRecord> filterEvents(List<ProvenanceEventRecord> provenanceEvents) {
    -        if(isFilteringEnabled()) {
    -            List<ProvenanceEventRecord> filteredEvents = new ArrayList<ProvenanceEventRecord>();
    +    private List<ProvenanceEventRecord> filterEvents(ComponentMapHolder componentMapHolder, List<ProvenanceEventRecord> provenanceEvents) {
    +        if (isFilteringEnabled()) {
    +            List<ProvenanceEventRecord> filteredEvents = new ArrayList<>();
     
                 for (ProvenanceEventRecord provenanceEventRecord : provenanceEvents) {
    -                if(!componentIds.isEmpty() && !componentIds.contains(provenanceEventRecord.getComponentId())) {
    -                    continue;
    +                final String componentId = provenanceEventRecord.getComponentId();
    +                if (!componentIds.isEmpty() && !componentIds.contains(componentId)) {
    +                    // If we aren't filtering it out based on component ID, let's see if this component has a parent process group IDs
    +                    // that is being filtered on
    +                    if (componentMapHolder == null) {
    +                        continue;
    +                    }
    +                    final String processGroupId = componentMapHolder.getProcessGroupId(componentId, provenanceEventRecord.getComponentType());
    +                    if (StringUtils.isEmpty(processGroupId)) {
    +                        continue;
    +                    }
    +                    // Check if any parent process group has the specified component ID
    +                    ParentProcessGroupSearchNode matchedComponent = componentMapHolder.getProcessGroupParent(componentId);
    --- End diff --
    
    `componentMapHolder.getProcessGroupParent(componentId)` will not work with RemoteInputPorts and RemoteOutputPorts.


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r158616024
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/provenance/ProvenanceEventConsumer.java ---
    @@ -218,18 +230,35 @@ private boolean isFilteringEnabled() {
             return componentTypeRegex != null || !eventTypes.isEmpty() || !componentIds.isEmpty();
         }
     
    -    private List<ProvenanceEventRecord> filterEvents(List<ProvenanceEventRecord> provenanceEvents) {
    -        if(isFilteringEnabled()) {
    -            List<ProvenanceEventRecord> filteredEvents = new ArrayList<ProvenanceEventRecord>();
    +    private List<ProvenanceEventRecord> filterEvents(ComponentMapHolder componentMapHolder, List<ProvenanceEventRecord> provenanceEvents) {
    +        if (isFilteringEnabled()) {
    +            List<ProvenanceEventRecord> filteredEvents = new ArrayList<>();
     
                 for (ProvenanceEventRecord provenanceEventRecord : provenanceEvents) {
    -                if(!componentIds.isEmpty() && !componentIds.contains(provenanceEventRecord.getComponentId())) {
    -                    continue;
    +                final String componentId = provenanceEventRecord.getComponentId();
    +                if (!componentIds.isEmpty() && !componentIds.contains(componentId)) {
    +                    // If we aren't filtering it out based on component ID, let's see if this component has a parent process group IDs
    +                    // that is being filtered on
    +                    if (componentMapHolder == null) {
    +                        continue;
    +                    }
    +                    final String processGroupId = componentMapHolder.getProcessGroupId(componentId, provenanceEventRecord.getComponentType());
    +                    if (StringUtils.isEmpty(processGroupId)) {
    +                        continue;
    +                    }
    +                    // Check if any parent process group has the specified component ID
    +                    ParentProcessGroupSearchNode matchedComponent = componentMapHolder.getProcessGroupParent(componentId);
    +                    while (matchedComponent != null && !matchedComponent.getId().equals(processGroupId) && !componentIds.contains(matchedComponent.getId())) {
    --- End diff --
    
    The condition `!matchedComponent.getId().equals(processGroupId)` should be removed.
    It does not work if a ProcessGroup id is used for filtering. For example, if there are Root, PG1, PG2, and Component C1 is in PG1, then the reporting task is configured to filter with PG2. In that case, `processGroupId` would be PG1. But it's not specified in `componentIds`. Since `componentIds` only contains PG2, C1 in PG1 should be filtered out. But the condition make C1 to pass.


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r157667259
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java ---
    @@ -640,7 +640,7 @@ private void consumeNiFiProvenanceEvents(ReportingContext context, NiFiFlow nifi
             final AnalysisContext analysisContext = new StandardAnalysisContext(nifiFlow, clusterResolvers,
                     // FIXME: This class cast shouldn't be necessary to query lineage. Possible refactor target in next major update.
                     (ProvenanceRepository)eventAccess.getProvenanceRepository());
    -        consumer.consumeEvents(eventAccess, context.getStateManager(), events -> {
    +        consumer.consumeEvents(null, eventAccess, context.getStateManager(), events -> {
    --- End diff --
    
    It would be more useful if we add another consumeEvents method signature (or replace existing one) having the last argument as `BiConsumer<ComponentMapHolder, List<ProvenanceEventRecord>>`.


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r157756093
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -174,34 +177,47 @@ public void onUnscheduled() {
             return properties;
         }
     
    -    private Map<String,String> createComponentMap(final ProcessGroupStatus status) {
    -        final Map<String,String> componentMap = new HashMap<>();
    +    private ComponentMapHolder createComponentMap(final ProcessGroupStatus status) {
    --- End diff --
    
    I agree, will move it there.


---

[GitHub] nifi issue #2351: NIFI-4707: S2SProvenanceReportingTask not returning correc...

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

    https://github.com/apache/nifi/pull/2351
  
    Thanks for the commit, great stuff!  Do you think the processing of the stack for each record will be ok in terms of performance impact?  I wonder if we'd be better off building a tree (basically a flow graph model) when building a component map, along with another "index" map from component id -> node in the tree. It might replace the need for other "inheritance" maps or property maps (as the node could hold the properties). Then we can get the component and traverse to the root during the filter?


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r158615663
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/provenance/ComponentMapHolder.java ---
    @@ -0,0 +1,120 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.nifi.reporting.util.provenance;
    +
    +import org.apache.nifi.controller.status.ConnectionStatus;
    +import org.apache.nifi.controller.status.PortStatus;
    +import org.apache.nifi.controller.status.ProcessGroupStatus;
    +import org.apache.nifi.controller.status.ProcessorStatus;
    +import org.apache.nifi.controller.status.RemoteProcessGroupStatus;
    +
    +import java.util.HashMap;
    +import java.util.Map;
    +
    +public class ComponentMapHolder {
    +    private static final String REMOTE_INPUT_PORT = "Remote Input Port";
    +    private static final String REMOTE_OUTPUT_PORT = "Remote Output Port";
    +    private final Map<String,String> componentNameMap = new HashMap<>();
    +    private final Map<String,ParentProcessGroupSearchNode> componentToParentGroupMap = new HashMap<>();
    +    private final Map<String,String> sourceToConnectionParentGroupMap = new HashMap<>();
    +    private final Map<String,String> destinationToConnectionParentGroupMap = new HashMap<>();
    +
    +    private ComponentMapHolder putAll(ComponentMapHolder holder) {
    +        this.componentNameMap.putAll(holder.componentNameMap);
    +        this.componentToParentGroupMap.putAll(holder.componentToParentGroupMap);
    +        this.sourceToConnectionParentGroupMap.putAll(holder.sourceToConnectionParentGroupMap);
    +        this.destinationToConnectionParentGroupMap.putAll(holder.destinationToConnectionParentGroupMap);
    +        return this;
    +    }
    +
    +    public String getComponentName(final String componentId) {
    +        return componentNameMap.get(componentId);
    +    }
    +
    +    public String getProcessGroupId(final String componentId, final String componentType) {
    +        // Where a Remote Input/Output Port resides is only available at ConnectionStatus.
    +        if (REMOTE_INPUT_PORT.equals(componentType)) {
    +            return destinationToConnectionParentGroupMap.get(componentId);
    +        } else if (REMOTE_OUTPUT_PORT.equals(componentType)) {
    +            return sourceToConnectionParentGroupMap.get(componentId);
    +        }
    +        ParentProcessGroupSearchNode parentNode = componentToParentGroupMap.get(componentId);
    +        return parentNode == null ? null : parentNode.getId();
    +    }
    +
    +    public ParentProcessGroupSearchNode getProcessGroupParent(final String componentId) {
    +        return componentToParentGroupMap.get(componentId);
    +    }
    +
    +    public static ComponentMapHolder createComponentMap(final ProcessGroupStatus status, final ParentProcessGroupSearchNode thisNode) {
    +        final ComponentMapHolder holder = new ComponentMapHolder();
    +        final Map<String,String> componentNameMap = holder.componentNameMap;
    +        final Map<String,ParentProcessGroupSearchNode> componentToParentGroupMap = holder.componentToParentGroupMap;
    +        final Map<String,String> sourceToConnectionParentGroupMap = holder.sourceToConnectionParentGroupMap;
    +        final Map<String,String> destinationToConnectionParentGroupMap = holder.destinationToConnectionParentGroupMap;
    +
    +        if (status != null) {
    +            ParentProcessGroupSearchNode parentNode = thisNode;
    +            componentNameMap.put(status.getId(), status.getName());
    +            // Put a root entry in if one does not yet exist
    +            if (parentNode == null) {
    +                parentNode = new ParentProcessGroupSearchNode(status.getId(), null);
    +                componentToParentGroupMap.put(status.getId(), parentNode);
    +            }
    +
    +            for (final ProcessorStatus procStatus : status.getProcessorStatus()) {
    +                componentNameMap.put(procStatus.getId(), procStatus.getName());
    +                componentToParentGroupMap.put(procStatus.getId(), new ParentProcessGroupSearchNode(status.getId(), parentNode));
    +            }
    +
    +            for (final PortStatus portStatus : status.getInputPortStatus()) {
    +                componentNameMap.put(portStatus.getId(), portStatus.getName());
    +                componentToParentGroupMap.put(portStatus.getId(), new ParentProcessGroupSearchNode(status.getId(), parentNode));
    +            }
    +
    +            for (final PortStatus portStatus : status.getOutputPortStatus()) {
    +                componentNameMap.put(portStatus.getId(), portStatus.getName());
    +                componentToParentGroupMap.put(portStatus.getId(), new ParentProcessGroupSearchNode(status.getId(), parentNode));
    +            }
    +
    +            for (final RemoteProcessGroupStatus rpgStatus : status.getRemoteProcessGroupStatus()) {
    +                componentNameMap.put(rpgStatus.getId(), rpgStatus.getName());
    +                componentToParentGroupMap.put(rpgStatus.getId(), new ParentProcessGroupSearchNode(status.getId(), parentNode));
    +            }
    +
    +            for (final ConnectionStatus connectionStatus : status.getConnectionStatus()) {
    +                componentNameMap.put(connectionStatus.getId(), connectionStatus.getName());
    +                componentToParentGroupMap.put(connectionStatus.getId(), new ParentProcessGroupSearchNode(status.getId(), parentNode));
    +                // Add source and destination for Remote Input/Output Ports because metadata for those are only available at ConnectionStatus.
    +                componentNameMap.computeIfAbsent(connectionStatus.getSourceId(), k -> connectionStatus.getSourceName());
    +                componentNameMap.computeIfAbsent(connectionStatus.getDestinationId(), k -> connectionStatus.getDestinationName());
    +                sourceToConnectionParentGroupMap.put(connectionStatus.getSourceId(), connectionStatus.getGroupId());
    +                destinationToConnectionParentGroupMap.put(connectionStatus.getDestinationId(), connectionStatus.getGroupId());
    +            }
    +
    +            for (final ProcessGroupStatus childGroup : status.getProcessGroupStatus()) {
    +                componentNameMap.put(childGroup.getId(), childGroup.getName());
    +                ParentProcessGroupSearchNode node = new ParentProcessGroupSearchNode(status.getId(), parentNode);
    +                componentToParentGroupMap.put(childGroup.getId(), node);
    --- End diff --
    
    ```
    ParentProcessGroupSearchNode node = new ParentProcessGroupSearchNode(status.getId(), parentNode);
    componentToParentGroupMap.put(childGroup.getId(), node);
    ```
    should be
    ```
    ParentProcessGroupSearchNode node = new ParentProcessGroupSearchNode(childGroup.getId(), parentNode);
    componentToParentGroupMap.put(childGroup.getId(), parentNode);
    ```


---

[GitHub] nifi issue #2351: NIFI-4707: S2SProvenanceReportingTask not returning correc...

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

    https://github.com/apache/nifi/pull/2351
  
    @mattyb149 Good catch. Yeah, simple ProcessGroupID tree structure would help optimizing lookup at filtering. Please add that and squash commits, then I'll do a final review and merge it. Thanks!


---

[GitHub] nifi issue #2351: NIFI-4707: S2SProvenanceReportingTask not returning correc...

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

    https://github.com/apache/nifi/pull/2351
  
    @mattyb149 I forgot to mention that I'm +1 with the rest of the code, so if my commit seems reasonable, please squash yours and merge it. Thanks!


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r157756243
  
    --- Diff: nifi-nar-bundles/nifi-atlas-bundle/nifi-atlas-reporting-task/src/main/java/org/apache/nifi/atlas/reporting/ReportLineageToAtlas.java ---
    @@ -640,7 +640,7 @@ private void consumeNiFiProvenanceEvents(ReportingContext context, NiFiFlow nifi
             final AnalysisContext analysisContext = new StandardAnalysisContext(nifiFlow, clusterResolvers,
                     // FIXME: This class cast shouldn't be necessary to query lineage. Possible refactor target in next major update.
                     (ProvenanceRepository)eventAccess.getProvenanceRepository());
    -        consumer.consumeEvents(eventAccess, context.getStateManager(), events -> {
    +        consumer.consumeEvents(null, eventAccess, context.getStateManager(), events -> {
    --- End diff --
    
    Agreed, I will update


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

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

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


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r158617030
  
    --- Diff: nifi-nar-bundles/nifi-extension-utils/nifi-reporting-utils/src/main/java/org/apache/nifi/reporting/util/provenance/ProvenanceEventConsumer.java ---
    @@ -218,18 +230,35 @@ private boolean isFilteringEnabled() {
             return componentTypeRegex != null || !eventTypes.isEmpty() || !componentIds.isEmpty();
         }
     
    -    private List<ProvenanceEventRecord> filterEvents(List<ProvenanceEventRecord> provenanceEvents) {
    -        if(isFilteringEnabled()) {
    -            List<ProvenanceEventRecord> filteredEvents = new ArrayList<ProvenanceEventRecord>();
    +    private List<ProvenanceEventRecord> filterEvents(ComponentMapHolder componentMapHolder, List<ProvenanceEventRecord> provenanceEvents) {
    +        if (isFilteringEnabled()) {
    +            List<ProvenanceEventRecord> filteredEvents = new ArrayList<>();
     
                 for (ProvenanceEventRecord provenanceEventRecord : provenanceEvents) {
    -                if(!componentIds.isEmpty() && !componentIds.contains(provenanceEventRecord.getComponentId())) {
    -                    continue;
    +                final String componentId = provenanceEventRecord.getComponentId();
    +                if (!componentIds.isEmpty() && !componentIds.contains(componentId)) {
    +                    // If we aren't filtering it out based on component ID, let's see if this component has a parent process group IDs
    +                    // that is being filtered on
    +                    if (componentMapHolder == null) {
    +                        continue;
    +                    }
    +                    final String processGroupId = componentMapHolder.getProcessGroupId(componentId, provenanceEventRecord.getComponentType());
    +                    if (StringUtils.isEmpty(processGroupId)) {
    +                        continue;
    +                    }
    +                    // Check if any parent process group has the specified component ID
    --- End diff --
    
    This comment does not sound right to me, if 'the specified component ID' means what user specified at ReportingTask filter property.


---

[GitHub] nifi pull request #2351: NIFI-4707: S2SProvenanceReportingTask not returning...

Posted by ijokarumawak <gi...@git.apache.org>.
Github user ijokarumawak commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/2351#discussion_r157666543
  
    --- Diff: nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteProvenanceReportingTask.java ---
    @@ -174,34 +177,47 @@ public void onUnscheduled() {
             return properties;
         }
     
    -    private Map<String,String> createComponentMap(final ProcessGroupStatus status) {
    -        final Map<String,String> componentMap = new HashMap<>();
    +    private ComponentMapHolder createComponentMap(final ProcessGroupStatus status) {
    --- End diff --
    
    This method probably should be in ProvenanceEventConsumer instead of each ReportingTask implementation so that other ReportingTasks can get benefits from it. How do you think?


---

[GitHub] nifi issue #2351: NIFI-4707: S2SProvenanceReportingTask not returning correc...

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

    https://github.com/apache/nifi/pull/2351
  
    @ijokarumawak Looks great, thanks!  Guess I was a little careless with implementation and testing on that go-round, thanks for getting it across the finish line. I ran the tests and with a number of scenarios on a live NiFi to verify things seem to be working smoothly. Will let Travis finish then merge to master.


---