You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2020/03/09 11:14:48 UTC

[GitHub] [nifi] simonbence opened a new pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

simonbence opened a new pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123
 
 
   Dear Community,
   
   This is a WIP pull request for the NIFI-7188. The feature is added and the majority of the tests are in place. One test set is missing, the one for ControllerSearchService, which will be added shortly. The changes contain 3 things:
   - Refactor in order to make the extension and testing simpler (mainly: splitting the search into smaller chunks with the same functionality)
   - A small fix related to the transmission status searching (due to the overlapping keywords, in some cases it returned with invalid result, now that is fixed and tested)
   - And the actual filters, affects mainly the following classes: ControllerSearchService, PropertyMatcher
   
   Thanks for your review, please do not hesitate to point out possible issues or derivation from community habits!
   
   Thank you for submitting a contribution to Apache NiFi.
   
   Please provide a short description of the PR here:
   
   #### Description of PR
   
   _Enables X functionality; fixes bug NIFI-YYYY._
   
   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?
   
   - [] 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.
   
   - [] Has your PR been rebased against the latest commit within the target branch (typically `master`)?
   
   - [] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] 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?
   - [ ] Have you verified that the full build is successful on both JDK 8 and JDK 11?
   - [ ] 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.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401883313
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,688 +16,222 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.Optional;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
+    private ComponentMatcher<ControllerServiceNode> matcherForControllerServiceNode;
 
     /**
-     * Searches term in the controller beginning from a given process group.
+     * Searches all parameter contexts and parameters.
      *
+     * @param searchQuery Details of the search
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultEnricher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            if (scope.getParent() != null) {
 
 Review comment:
   Why is the root process group skipped here?
   Due to this, it is not possible to search for the root group's name/id or for variables defined at the root level.
   Though it did not work in the original version either (as far as I see), I think it should.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400929270
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
 
 Review comment:
   ```suggestion
   *transmitting*: Adds Remote Process Groups to the result list that are transmitting data at the time.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390380984
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/BasicAttributeMatcher.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class BasicAttributeMatcher<T extends Connectable> implements AttributeMatcher<T> {
 
 Review comment:
   Usually it's better to avoid boolean flags (especially if they have no business logic implications).
   I'm wondering if it wouldn't be better to use polymorphism instead, like this:
   ```java
   public class BasicAttributeMatcher<T extends Connectable> implements AttributeMatcher<T> {
       private static final String LABEL_ID = "Id";
       private static final String LABEL_VERSION_CONTROL_ID = "Version Control ID";
   
       @Override
       public void match(final T component, final SearchQuery query, final List<String> matches) {
           final String searchTerm = query.getTerm();
   
           addIfMatching(searchTerm, component.getIdentifier(), LABEL_ID, matches);
           addIfMatching(searchTerm, component.getVersionedComponentId().orElse(null), LABEL_VERSION_CONTROL_ID, matches);
       }
   }
   
   public class ExtendedAttributeMatcher<T extends Connectable> extends BasicAttributeMatcher<T> {
       private static final String LABEL_NAME = "Name";
       private static final String LABEL_COMMENTS = "Comments";
   
       @Override
       public void match(final T component, final SearchQuery query, final List<String> matches) {
           super.match(component, query, matches);
   
           final String searchTerm = query.getTerm();
   
           addIfMatching(searchTerm, component.getName(), LABEL_NAME, matches);
           addIfMatching(searchTerm, component.getComments(), LABEL_COMMENTS, matches);
       }
   }
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r402151889
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,688 +16,222 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.Optional;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
+    private ComponentMatcher<ControllerServiceNode> matcherForControllerServiceNode;
 
     /**
-     * Searches term in the controller beginning from a given process group.
+     * Searches all parameter contexts and parameters.
      *
+     * @param searchQuery Details of the search
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultEnricher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            if (scope.getParent() != null) {
 
 Review comment:
   This came from the original implementation. In this PR I strived to keep the existing functionality the same in most cases so I would find it better to change this under a separate ticket. However, I find it reasonable to allow users to search in the variable registry for root process group as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391121589
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/ComponentMatcher.java
 ##########
 @@ -0,0 +1,28 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import javax.annotation.Nonnull;
+import java.util.Optional;
+
+    public interface ComponentMatcher<COMPONENT_TYPE> {
 
 Review comment:
   Indentation

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390923477
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
 
 Review comment:
   Good highlighting. I did not notice most probably due I did not start from stretch but I applied refactors step-by-step.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393027204
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
 
 Review comment:
   Not sure I understand. For me this seems to work:
   ```java
               flowController.getFlowManager()
                   .getParameterContextManager()
                   .getParameterContexts()
                   .stream()
                   .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, searchQuery.getUser()))
                   .forEach(parameterContext -> {
                       final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getParameterResultEnricher(parameterContext);
   
                       matcherForParameterContext.match(parameterContext, searchQuery)
                           .ifPresent(match -> results.getParameterContextResults().add(match));
   
                       parameterContext.getParameters().values().stream()
                           .map(component -> matcherForParameter.match(component, searchQuery))
                           .filter(Optional::isPresent)
                           .map(result -> resultEnricher.enrich(result.get()))
                           .forEach(result -> results.getParameterResults().add(result));
                   });
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400947528
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. If provided, the only groups will be searched are the ones containing the filter value in their name or id. This is transitive, so if any parent Process Group is fit, than the contained Process Groups are subject of the search. In case on non-existing group, the filter will be used but the result list will be empty.
+
+*properties*: In case of valid value, this filter excludes the findings would be added based on property. The valid values are: "_no_", "_none_", "_false_", "_exclude_" and "_0_". Every other value is invalid, resulting that the search will ignore the filter.
 
 Review comment:
   ```suggestion
   *properties*: With this, users can prevent property matches to appear in the search result. Valid values are: "no", "none", "false", "exclude" and "0".
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400954001
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. If provided, the only groups will be searched are the ones containing the filter value in their name or id. This is transitive, so if any parent Process Group is fit, than the contained Process Groups are subject of the search. In case on non-existing group, the filter will be used but the result list will be empty.
+
+*properties*: In case of valid value, this filter excludes the findings would be added based on property. The valid values are: "_no_", "_none_", "_false_", "_exclude_" and "_0_". Every other value is invalid, resulting that the search will ignore the filter.
+
+=== Search examples
+
+==== Example 1
+
+`group:myGroup processor1`
+
+The search query will result a list of components are containing `processor1` in their relevant attributes. The search will be executed under Process Groups (directly or via contained Process Groups) containing the string "myGroup" in their name or id.
+
+==== Example 2
+
+`scope:here properties:exclude import`
+
+The example search query above returns with every component is contained (directly or via contained Process Groups) by the currently active Process Group, where the component's relevant attributes (name, comment, etc.) contain the word _import_ but excludes the findings based on property attributes.
 
 Review comment:
   ```suggestion
   Search will be restricted to the currently active process group (and process groups within that).
   The result will contain components that match for "import" but property matches will be excluded.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393211248
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/ComponentSearchResultEnricher.java
 ##########
 @@ -0,0 +1,33 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+
+/**
+ * Responsible for enriching the query result based on the containing component (for example process group).
+ */
+public interface ComponentSearchResultEnricher {
+    /**
+     * Enriches the incoming result object. Might change it or fill up one or more empty parameter.
 
 Review comment:
    Minor suggestion: `set` instead of `fill up` may be better. Also `unset` instead of `empty` may be better.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392338390
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/ScheduledStateMatcher.java
 ##########
 @@ -0,0 +1,57 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public class ScheduledStateMatcher implements AttributeMatcher<ProcessorNode>  {
+    private static final String SEARCH_TERM_DISABLED = "disabled";
+    private static final String SEARCH_TERM_INVALID = "invalid";
+    private static final String SEARCH_TERM_VALIDATING = "validating";
+    private static final String SEARCH_TERM_RUNNING = "running";
+    private static final String SEARCH_TERM_STOPPED = "stopped";
+
+    private static final String MATCH_PREFIX = "Run status: ";
+    private static final String MATCH_DISABLED = "Disabled";
+    private static final String MATCH_INVALID = "Invalid";
+    private static final String MATCH_VALIDATING = "Validating";
+    private static final String MATCH_RUNNING = "Running";
+    private static final String MATCH_STOPPED = "Stopped";
+
+    @Override
+    public void match(final ProcessorNode component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (ScheduledState.DISABLED.equals(component.getScheduledState()) && StringUtils.containsIgnoreCase(SEARCH_TERM_DISABLED, searchTerm)) {
 
 Review comment:
   This if-else branching is not logically the same as the original.
   For example in your case the second branch might get evaluated if the scheduled state is `DISABLED` but there's no `"disabled"` (or something like that) in the search term.
   
   With the original logic it was not the case - If the component was `DISABLED`, the other cases did not get evaluated period.
   
   Doesn't seem to be too significant but maybe (not sure) _disabled_ components _can_ be _invalid_ (and/or _stopped_) at the same time, and you probably don't want those to come up when you are looking for invalid/stopped components in general.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390923477
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
 
 Review comment:
   Good highlighting. I did not notice most probably due I did not start from stretch but I applied refactors step-by-step.
   
   Update: as the full code piece covers search on both `ParameterContext` and `Parameter` types and the later references to the containing context, it do not end up nice if I try to merge them. I left it as it was.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392215392
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultErnicher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
 Review comment:
   Typo: groupResultEnricher

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390915238
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
 
 Review comment:
   It might makes sense but I do not know if it is a realistic use case. But from development perspective it does not make a great effort.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392362628
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultErnicher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, groupResultErnicher, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, resultEnricher, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, resultEnricher, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, resultEnricher, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, resultEnricher, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, resultEnricher, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, resultEnricher, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, resultEnricher, results.getLabelResults());
         }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
+    }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return !searchQuery.hasFilter(FILTER_NAME_GROUP) || eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP));
     }
 
     /**
-     * Searches all parameter contexts and parameters
-     * @param results Search results
-     * @param search  The search term
+     * Check is the group is eligible for the filter value. It might be eligible based on name or id.
+     *
+     * @param scope The subject process group.
+     * @param filterValue The value to match against.
+     *
+     * @return True in case the scope process group or any parent is matching. A group is matching when it's name or it's id contains the filter value.
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<ProcessGroup> lineage = getLineage(scope);
 
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
+        for (final ProcessGroup group : lineage) {
+            if (StringUtils.containsIgnoreCase(group.getName(), filterValue) || StringUtils.containsIgnoreCase(group.getIdentifier(), filterValue)) {
+                return true;
             }
         }
 
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
+    private List<ProcessGroup> getLineage(final ProcessGroup group) {
+        final LinkedList<ProcessGroup> result = new LinkedList<>();
+        result.add(group);
+        ProcessGroup current = group;
 
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
+        while (current.getParent() != null) {
+            current = current.getParent();
+            result.addLast(current);
         }
 
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private <T extends Authorizable> void searchComponentType(
+               final Collection<T> components,
+               final NiFiUser user,
+               final SearchQuery searchQuery,
+               final ComponentMatcher<T> matcher,
+               final ComponentSearchResultEnricher matchEnriching,
 
 Review comment:
   matchEnriching => (result)Enricher ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392237603
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/GeneralComponentSearchResultEnricher.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+
+public class GeneralComponentSearchResultEnricher extends AbstractComponentSearchResultEnricher {
+    public GeneralComponentSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        super(processGroup, user, authorizer);
+    }
+
+    @Override
+    public ComponentSearchResultDTO enrich(final ComponentSearchResultDTO input) {
+        input.setGroupId(processGroup.getIdentifier());
+        input.setParentGroup(buildResultGroup(processGroup.getParent(), user));
 
 Review comment:
   Should we use `processGroup` itself here instead of `processGroup.getParent()`?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390925256
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/ComponentMatcherFactory.java
 ##########
 @@ -0,0 +1,78 @@
+/*
+ * 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.web.search;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.web.search.attributematchers.AttributeMatcher;
+
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ComponentMatcherFactory {
+    public ComponentMatcher<Connectable> getInstanceForConnectable(final List<AttributeMatcher<Connectable>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<Connection> getInstanceForConnection(final List<AttributeMatcher<Connection>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), new GetConnectionName());
+    }
+
+    public ComponentMatcher<Parameter> getInstanceForParameter(final List<AttributeMatcher<Parameter>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getDescriptor().getName(), component -> component.getDescriptor().getName());
+    }
+
+    public ComponentMatcher<ParameterContext> getInstanceForParameterContext(final List<AttributeMatcher<ParameterContext>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<ProcessGroup> getInstanceForProcessGroup(final List<AttributeMatcher<ProcessGroup>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<RemoteProcessGroup> getInstanceForRemoteProcessGroup(final List<AttributeMatcher<RemoteProcessGroup>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    private static class GetConnectionName implements Function<Connection, String> {
+        private static final String DEFAULT_NAME_PREFIX = "From source ";
+        private static final String SEPARATOR = ", ";
+
+        public String apply(final Connection component) {
+            String result = null;
+
+            if (StringUtils.isNotBlank(component.getName())) {
+                result = component.getName();
+            } else if (!component.getRelationships().isEmpty()) {
+                result = component.getRelationships().stream() //
 
 Review comment:
   Valid point. However it looks somewhat harder to read

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401756847
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+Users can use pre-defined (case-insensitive) keywords in the search box that will check certain conditions (instead of trying to match labels and values etc.). Keywords can be used with filters (see below) but not with other search terms (otherwise they won't be treated as keywords) and only one keyword can be used at a time. Note however that keywords will also be treated as general search terms at the same time. The supported keywords are the following:
+
+*back pressure*: Adds Connections to the result list that are applying back pressure at the time.
 
 Review comment:
   Please group the keywords based in their usage instead of the alphabetical order. Eg.:
   Scheduling Strategy
   
   - event
   - timer

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390921644
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
 
 Review comment:
   My experience is that it helps prevent automatic formatters to break it in a way makes it hard to read, but I do not insist to go on with anything like this. If the community is against that, I am okay with it and I will remove them.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393256923
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenContentOfTheFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
 
 Review comment:
   The parameter could be removed and replaced with a local variable:
   `final Collection<String> searchedProcessGroups = processGroups.keySet();`
   With that the name could be changed to `thenContentOfGroupsAreSearched`.
   
   However not even sure how this method proves that the groups are `searched`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391108275
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
 ##########
 @@ -3106,8 +3106,8 @@ public ProvenanceEventDTO submitReplay(final Long eventId) {
     // -----------------------------------------
 
     @Override
-    public SearchResultsDTO searchController(final String query) {
-        return controllerFacade.search(query);
+    public SearchResultsDTO searchController(final String query, final String activeGroupId) {
 
 Review comment:
   Minor inconsistency: This parameter is named `currentGroupId` in the interface.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] asfgit closed pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390455730
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -147,8 +148,8 @@
     // properties
     private NiFiProperties properties;
     private DtoFactory dtoFactory;
-    private VariableRegistry variableRegistry;
     private ControllerSearchService controllerSearchService;
+    private SearchQueryParser searchQueryParser;
 
 Review comment:
   As far as responsibilities go, `SearchQueryParser` seems to be something that should be in the `ControllerSearchService` instead of the `ControllerFacade`.
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393216459
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/ComponentMatcher.java
 ##########
 @@ -0,0 +1,41 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Optional;
+
+/**
+ * Service responsible to clamp all the possible matches for a given component type.
+ *
+ * @param <COMPONENT_TYPE> The component type.
+ */
+public interface ComponentMatcher<COMPONENT_TYPE> {
 
 Review comment:
   I think in general the current javadoc for this interface is more suited for the `AttributeBasedComponentMatcher`.
   This interface doesn't specify collections of matches or matchers and it doesn't need to.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390544344
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/AttributeMatcher.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public interface AttributeMatcher<T> {
+    String SEPARATOR = ": ";
+
+    void match(T component, SearchQuery query, List<String> matches);
+
+    static void addIfMatching(final String searchTerm, final String subject, final String label, final List<String> matches) {
+        final String match = (label == null) //
 
 Review comment:
   Is there a reason to start considering `label == null` a valid case?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392225272
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
 
 Review comment:
   Typo: double spaces

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390903549
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
 
 Review comment:
   You are perfectly correct, I did not consider this.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390908556
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
 
 Review comment:
   Your observation is correct, it is a mistake on my account and I will fix it. As for the name, I prefer the scopeMatchEnricher. My original intention was to communicate that it only partially enriches, but as in the end it's used in one place only, it makes sense to give it a more specific name.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401759935
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+Users can use pre-defined (case-insensitive) keywords in the search box that will check certain conditions (instead of trying to match labels and values etc.). Keywords can be used with filters (see below) but not with other search terms (otherwise they won't be treated as keywords) and only one keyword can be used at a time. Note however that keywords will also be treated as general search terms at the same time. The supported keywords are the following:
+
+*back pressure*: Adds Connections to the result list that are applying back pressure at the time.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds Connections to the result list that contain expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds Processors to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list that are not transmitting data at the time.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors to the result list that are set to run on the primary node only (whether if the Processor is currently running or not).
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds Processors to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list that are transmitting data at the time.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list that are validating at the time.
+
+=== Filters
+
+Filters can be added to the search box as key-value pairs where the keys are predefined and check certain conditions based on the given value. The syntax is "key:value". Filters can be used together with other search terms and multiple filters can be used. The only constraint is that the search must start with the filters. Unknown filters or known filters with unknown values are ignored. If the same filter key appears multiple times, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "here". The usage of this filter looks like "scope:here". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. Search will be restricted to groups (and their components - including subgroups and their components) the names or ids of which match the filter value. If no group matches the filter the result list will be empty.
+
+*properties*: With this, users can prevent property matches to appear in the search result. Valid values are: "no", "none", "false", "exclude" and "0".
+
+=== Search examples
+
+==== Example 1
+
+`group:myGroup processor1`
 
 Review comment:
   Could you please add some more basic examples too?
   Like: simple search, keyword only search.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400930004
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
 
 Review comment:
   ```suggestion
   *validating*: Adds Processors to the result list that are validating at the time.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391169992
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/BackPressureAttributeMatcher.java
 ##########
 @@ -0,0 +1,62 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.processor.DataUnit;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+public class BackPressureAttributeMatcher implements AttributeMatcher<Connection> {
+    private static final String MATCH_PREFIX_SIZE = "Back pressure data size: ";
+    private static final String MATCH_PREFIX_COUNT = "Back pressure count: ";
+    private static final Set<String> KEYWORDS = new HashSet<>(Arrays.asList( //
+            "back pressure", //
+            "pressure"));
+
+    @Override
+    public void match(final Connection component, final SearchQuery query, final List<String> matches) {
+        if (containsKeyword(query)) {
+            final String backPressureDataSize = component.getFlowFileQueue().getBackPressureDataSizeThreshold();
+            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
+            final long backPressureCount = component.getFlowFileQueue().getBackPressureObjectThreshold();
+
+            if (backPressureBytes > 0) {
+                matches.add(MATCH_PREFIX_SIZE + backPressureDataSize);
+            }
+
+            if (backPressureCount > 0) {
+                matches.add(MATCH_PREFIX_COUNT + backPressureCount);
+            }
+        }
+    }
+
+    private boolean containsKeyword(final SearchQuery query) {
+        for (final String keyword : KEYWORDS) {
 
 Review comment:
   Minor suggestion: `stream().anyMatch(...)`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393259703
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenContentOfTheFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        int numberOfCheckedGroups = 0;
+
+        for (final String processGroup : searchedProcessGroups) {
+            // Checking on funnels is arbitrary, any given component we expect to be searched would be a good candidate
+            Mockito.verify(processGroups.get(processGroup), Mockito.times(1)).getFunnels();
+            numberOfCheckedGroups++;
+        }
+
+        Assert.assertEquals(searchedProcessGroups.size(), numberOfCheckedGroups);
 
 Review comment:
   What is the purpose of this assertion?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401907806
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/AbstractComponentSearchResultEnricher.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
+
+abstract class AbstractComponentSearchResultEnricher implements ComponentSearchResultEnricher {
+    protected final ProcessGroup processGroup;
+    protected final NiFiUser user;
+    protected final Authorizer authorizer;
+
+    AbstractComponentSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        this.processGroup = processGroup;
+        this.user = user;
+        this.authorizer = authorizer;
+    }
+
+    /**
+     * Builds the nearest versioned parent result group for a given user.
+     *
+     * @param group The containing group
+     * @param user The current NiFi user
+     * @return Versioned parent group
+     */
+    protected SearchResultGroupDTO buildVersionedGroup(final ProcessGroup group, final NiFiUser user) {
+        if (group == null) {
+            return null;
+        }
+
+        ProcessGroup tmpParent = group.getParent();
+        ProcessGroup tmpGroup = group;
+
+        // search for a versioned group by traversing the group tree up to the root
+        while (!tmpGroup.isRootGroup()) {
+            if (tmpGroup.getVersionControlInformation() != null) {
+                return buildResultGroup(tmpGroup, user);
+            }
+
+            tmpGroup = tmpParent;
+            tmpParent = tmpGroup.getParent();
+        }
 
 Review comment:
   Just an idea but it would be more straightforward for me in this way:
   ```
           ProcessGroup current = group;
   
           // search for a versioned group by traversing the group tree up to the root
           while (!current.isRootGroup()) {
               if (current.getVersionControlInformation() != null) {
                   return buildResultGroup(current, user);
               }
   
               current = current.getParent();
           }
   ```
   In this case `ComponentSearchResultEnricherTest` line 73 needs to be removed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390351630
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/ComponentMatcherFactory.java
 ##########
 @@ -0,0 +1,78 @@
+/*
+ * 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.web.search;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
+import org.apache.nifi.parameter.Parameter;
+import org.apache.nifi.parameter.ParameterContext;
+import org.apache.nifi.web.search.attributematchers.AttributeMatcher;
+
+import java.util.List;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+public class ComponentMatcherFactory {
+    public ComponentMatcher<Connectable> getInstanceForConnectable(final List<AttributeMatcher<Connectable>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<Connection> getInstanceForConnection(final List<AttributeMatcher<Connection>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), new GetConnectionName());
+    }
+
+    public ComponentMatcher<Parameter> getInstanceForParameter(final List<AttributeMatcher<Parameter>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getDescriptor().getName(), component -> component.getDescriptor().getName());
+    }
+
+    public ComponentMatcher<ParameterContext> getInstanceForParameterContext(final List<AttributeMatcher<ParameterContext>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<ProcessGroup> getInstanceForProcessGroup(final List<AttributeMatcher<ProcessGroup>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    public ComponentMatcher<RemoteProcessGroup> getInstanceForRemoteProcessGroup(final List<AttributeMatcher<RemoteProcessGroup>> attributeMatchers) {
+        return new AttributeBasedComponentMatcher<>(attributeMatchers, component -> component.getIdentifier(), component -> component.getName());
+    }
+
+    private static class GetConnectionName implements Function<Connection, String> {
+        private static final String DEFAULT_NAME_PREFIX = "From source ";
+        private static final String SEPARATOR = ", ";
+
+        public String apply(final Connection component) {
+            String result = null;
+
+            if (StringUtils.isNotBlank(component.getName())) {
+                result = component.getName();
+            } else if (!component.getRelationships().isEmpty()) {
+                result = component.getRelationships().stream() //
 
 Review comment:
   Maybe slightly simpler:
   ```java
                   result = connection.getRelationships().stream()
                           .map(Relationship::getName)
                           .filter(StringUtils::isNotBlank)
                           .collect(Collectors.joining(SEPARATOR));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r397059898
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceIntegrationTest.java
 ##########
 @@ -0,0 +1,598 @@
+/*
+ * 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.web.controller;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.web.controller.ComponentMockUtil.getBasicRelationships;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getChildProcessGroup;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getConnection;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getFunnel;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getPort;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getProcessorNode;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getPublicPort;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getRemoteProcessGroup;
+
+public class ControllerSearchServiceIntegrationTest extends AbstractControllerSearchIntegrationTest {
 
 Review comment:
   `.validate(results)` call is missing in multiple places. Please check the tests again.
   (I added a comment for one occasion but there are several.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400928274
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
 
 Review comment:
   ```suggestion
   *primary:* Adds Processors to the result list that are set to run on the primary node only (whether the Processor is currently running or not).
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396097323
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenContentOfTheFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
 
 Review comment:
   In some cases, all the groups are searched, in those scenarios the `processGrooups.keySet()` might be used. In other cases, not all the groups are searched (because of authorization), that is why in some test method there is an explicit list of groups.
   
   I did assume that during search, the getter for a given set (collection) component will be called. Now that I look the code second time, I found a way looks better: I remained with funnel (this is a perfectly arbitrary pick). Ever group has one funnel. In the reworked version, the test checks if the funnel for the given group (which we expect to be searched) was checked with the funnel matcher. I actually found a wrong expectation from the previous version, thanks! What is your opinion?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390910512
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
 
 Review comment:
   In this case I leaning towards to simplify based on your suggestion.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400958822
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. If provided, the only groups will be searched are the ones containing the filter value in their name or id. This is transitive, so if any parent Process Group is fit, than the contained Process Groups are subject of the search. In case on non-existing group, the filter will be used but the result list will be empty.
+
+*properties*: In case of valid value, this filter excludes the findings would be added based on property. The valid values are: "_no_", "_none_", "_false_", "_exclude_" and "_0_". Every other value is invalid, resulting that the search will ignore the filter.
+
+=== Search examples
+
+==== Example 1
+
+`group:myGroup processor1`
+
+The search query will result a list of components are containing `processor1` in their relevant attributes. The search will be executed under Process Groups (directly or via contained Process Groups) containing the string "myGroup" in their name or id.
+
+==== Example 2
+
+`scope:here properties:exclude import`
+
+The example search query above returns with every component is contained (directly or via contained Process Groups) by the currently active Process Group, where the component's relevant attributes (name, comment, etc.) contain the word _import_ but excludes the findings based on property attributes.
+
+==== Example 3
+
+`scope:here invalid`
+
+Executing this search will result the list of invalid Processors and Ports contained by the current group. The search execution will also use "invalid" as search term with the same scope.
 
 Review comment:
   ```suggestion
   Search will be restricted to the currently selected process group (and its sub process groups).
   "invalid" here (as it is alone after the filter) will be treated both as a keyword and a regular search term.
   The result will contain invalid Processors and Ports as well as all other components that match for "invalid" in any way.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390517409
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
 
 Review comment:
   Minor suggestion: The ternary operator expressions are usually harder to understand as those with binary ones. Maybe we could use
   ```
   return !searchQuery.hasFilter(FILTER_NAME_GROUP) || eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP));
   ```
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390927648
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/MatchEnriching.java
 ##########
 @@ -0,0 +1,56 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+public class MatchEnriching implements Function<ComponentSearchResultDTO, ComponentSearchResultDTO> {
 
 Review comment:
   I am not sure how about the customs in the project regarding (but I guess you pointed out this because it is not the usual way), so I cannot really oppose :) My primary intention with this is to provide an interface to able to depend on it if necessary. So if it is a fitting approach, I would introduce an interface which would "replace" `Function` here.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396090029
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -1614,15 +1615,20 @@ private void setComponentDetails(final ProvenanceEventDTO dto) {
     /**
      * Searches this controller for the specified term.
      *
-     * @param search search
+     * @param searchLiteral search
+     * @param activeGroupId the identifier of the currently visited group
      * @return result
      */
-    public SearchResultsDTO search(final String search) {
+    public SearchResultsDTO search(final String searchLiteral, final String activeGroupId) {
         final ProcessGroup rootGroup = getRootGroup();
+        final ProcessGroup activeGroup =  flowController.getFlowManager().getGroup(activeGroupId);
 
 Review comment:
   I added this a couple of days ago, I just forgot to answer here. Thanks for pointing out! I am just adding a unit test to ensure about it's behaviour.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401751487
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/ComponentVariableRegistryAttributeMatcher.java
 ##########
 @@ -0,0 +1,44 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+import java.util.Map;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class ComponentVariableRegistryAttributeMatcher implements AttributeMatcher<ProcessGroup> {
 
 Review comment:
   `VariableRegistryMatcher` would be a more compact name but still descriptive enough.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400964323
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
 
 Review comment:
   ```suggestion
   Filters can be added to the seatch box as key-value pairs where the keys are predifined and check certain conditions based on the given value.
   The syntax is "key:value".
   Filters can be used together with other search terms and multiple filters can be used. The only constraint is that the search must start with the filters.
   Unknown filters or known filters with unknown values are ignored. If the same filter key appears multiple times, the first will be used. The order of different filters has no effect on the result.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390513175
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
+            if (groupName.contains(filterValue)) {
+                return true;
             }
         }
 
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getLineage(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
 
 Review comment:
   Minor: `current` feels a better name instead of `actual`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390546189
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/AttributeMatcher.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public interface AttributeMatcher<T> {
+    String SEPARATOR = ": ";
+
+    void match(T component, SearchQuery query, List<String> matches);
+
+    static void addIfMatching(final String searchTerm, final String subject, final String label, final List<String> matches) {
+        final String match = (label == null) //
+                ? subject //
+                : new StringBuilder(label).append(SEPARATOR).append(subject).toString();
 
 Review comment:
   `label + ": " + subject` is perfectly fine, `StringBuilder` will be used automatically under the hood.
   Its explicit use becomes important when used in a separate block (like in a `for` cycle for example).

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392336703
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/PropertyMatcher.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class PropertyMatcher implements AttributeMatcher<ProcessorNode> {
+    private static final String LABEL_NAME = "Property name";
+    private static final String LABEL_VALUE = "Property value";
+    private static final String LABEL_DESCRIPTION = "Property description";
+
+    private final static String FILTER_NAME_PROPERTIES = "properties";
+    private final static Set<String> FILTER_VALUES_PROPERTIES_EXCLUSION = new HashSet<>(Arrays.asList("no", "none", "false", "exclude", "0"));
+
+    @Override
+    public void match(final ProcessorNode component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (!propertiesAreFilteredOut(query)) {
+            for (final Map.Entry<PropertyDescriptor, String> entry : component.getRawPropertyValues().entrySet()) {
+                final PropertyDescriptor descriptor = entry.getKey();
+                addIfMatching(searchTerm, descriptor.getName(), LABEL_NAME, matches);
+                addIfMatching(searchTerm, descriptor.getDescription(), LABEL_DESCRIPTION, matches);
+
+                // never include sensitive properties values in search results
+                if (!descriptor.isSensitive()) {
+                    final String value = (entry.getValue() == null)
+                        ? descriptor.getDefaultValue()
+                        : entry.getValue();
+
+                    // evaluate if the value matches the search criteria
+                    if (StringUtils.containsIgnoreCase(value, searchTerm)) {
+                        matches.add(new StringBuilder(LABEL_VALUE).append(SEPARATOR).append(descriptor.getName()).append(" - ").append(value).toString());
 
 Review comment:
   Not sure I understand why this was resolved. Or a later commit will resolve this? (I think we should mark comments resolved when the change has been submitted, otherwise it becomes hard to follow.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393206375
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/SearchQueryParser.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+/**
+ * Service responsible to translate incoming user and contextual information.
 
 Review comment:
   Service responsible _for translating_

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400945077
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. If provided, the only groups will be searched are the ones containing the filter value in their name or id. This is transitive, so if any parent Process Group is fit, than the contained Process Groups are subject of the search. In case on non-existing group, the filter will be used but the result list will be empty.
 
 Review comment:
   ```suggestion
   *group*: This filter narrows the scope of the search based on the provided Process Group name or id. Search will be restricted to groups (and their components - including subgroups and their components) the names or ids of which match the filter value. If no group matches the filter the result list will be empty.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392216123
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -1614,15 +1615,20 @@ private void setComponentDetails(final ProvenanceEventDTO dto) {
     /**
      * Searches this controller for the specified term.
      *
-     * @param search search
+     * @param searchLiteral search
+     * @param activeGroupId the identifier of the currently visited group
      * @return result
      */
-    public SearchResultsDTO search(final String search) {
+    public SearchResultsDTO search(final String searchLiteral, final String activeGroupId) {
         final ProcessGroup rootGroup = getRootGroup();
+        final ProcessGroup activeGroup =  flowController.getFlowManager().getGroup(activeGroupId);
 
 Review comment:
   Typo: double spaces

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390510434
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
 
 Review comment:
   The `group:xxx` filter can only refer to group names? I think it would be useful to allow UUID as well.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392223836
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
 
 Review comment:
   It is not the term only but the preprocessed query string as a query object, isn't it?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401209373
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+Users can use pre-defined (case-insensitive) keywords in the search box that will check certain conditions (instead of trying to match labels and values etc.). Keywords can be used with filters (see below) but not with other search terms (otherwise they won't be treated as keywords) and only one keyword can be used at a time. Note however that keywords will also be treated as general search terms at the same time. The supported keywords are the following:
+
+*back pressure*: Adds Connections to the result list that are applying back pressure at the time.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds Connections to the result list that contain expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds Processors to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list that are not transmitting data at the time.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors to the result list that are set to run on the primary node only (whether if the Processor is currently running or not).
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds Processors to the result list where the Scheduling Strategy is "Event Timer".
 
 Review comment:
   The right name is "Timer Driven".

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391066175
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
+            if (groupName.contains(filterValue)) {
+                return true;
             }
         }
 
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getLineage(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(parameterContext.getIdentifier());
-        dto.setName(parameterContext.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchString, final Parameter parameter) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameter.getDescriptor().getName(), "Name", matches);
-        addIfAppropriate(searchString, parameter.getDescriptor().getDescription(), "Description", matches);
-        if (!parameter.getDescriptor().isSensitive()) {
-            addIfAppropriate(searchString, parameter.getValue(), "Value", matches);
-        }
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
+                .getParameterContextManager() //
+                .getParameterContexts()
+                .stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, searchQuery.getUser())) //
+                .collect(Collectors.toSet());
 
-        if (matches.isEmpty()) {
-            return null;
+        for (final ParameterContext parameterContext : parameterContexts) {
+            final SearchResultGroupDTO parentGroup = new SearchResultGroupDTO();
+            parentGroup.setId(parameterContext.getIdentifier());
+            parentGroup.setName(parameterContext.getName());
+            final MatchEnriching parameterMatchEnriching = new MatchEnriching(Optional.empty(), Optional.of(parentGroup), Optional.empty());
+
+            matcherForParameterContext.match(parameterContext, searchQuery).ifPresent(match -> results.getParameterContextResults().add(match));
 
 Review comment:
   Shouldn't the parameterContext match get enriched as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391123308
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/AttributeMatcher.java
 ##########
 @@ -0,0 +1,38 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public interface AttributeMatcher<T> {
+    String SEPARATOR = ": ";
+
+    void match(T component, SearchQuery query, List<String> matches);
+
+    static void addIfMatching(final String searchTerm, final String subject, final String label, final List<String> matches) {
+        final String match = (label == null) //
 
 Review comment:
   Not in the happy path cases

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392371549
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/ComponentMatcher.java
 ##########
 @@ -0,0 +1,43 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import javax.annotation.Nonnull;
+import java.util.Optional;
+
+/**
+ * Service responsible to clamp all the possible matches for a given component type.
+ *
+ * @param <COMPONENT_TYPE> The component type.
+ */
+public interface ComponentMatcher<COMPONENT_TYPE> {
+
+    /**
+     * Tries to match the incoming search query against a given component with all the matchers are applicable for the given type.
+     *
+     * @param component The component to match against.
+     * @param query The search query to match.
+     *
+     * @return The result of the matching. {@link Optional#empty()} if there was no match, contains {@link ComponentSearchResultDTO}
+     * with the details of the results in case there was at least one match for the given component and query.
+     */
+    @Nonnull
 
 Review comment:
   I haven't seen it used in this project. Though I don't know if it is a convention or not. JSR305 comes in only via a transitive dependency, so I'd say it is intentionally not added / used.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396091351
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/SearchQuery.java
 ##########
 @@ -0,0 +1,72 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+/**
+ * Represents the data set the search query executes based on.
+ */
+public interface SearchQuery {
+
+    /**
+     * The part of the query string not containing metadata (filters).
+     *
+     * @return The query string used for executing the search.
+     */
+    String getTerm();
+
+    /**
+     * Returns if the query contains a given filter (regardless the value).
 
 Review comment:
   Thanks. Yeah, the "true" was missing. Fixed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392334876
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -147,8 +148,8 @@
     // properties
     private NiFiProperties properties;
     private DtoFactory dtoFactory;
-    private VariableRegistry variableRegistry;
     private ControllerSearchService controllerSearchService;
+    private SearchQueryParser searchQueryParser;
 
 Review comment:
   Okay, that's fair.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390493353
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
 
 Review comment:
   This is used for scope (only). According to the original logic, in it's case the `parentGroup` should be built with the parent of `scope` (see
   ```java
   groupMatch.setParentGroup(buildResultGroup(>>>group.getParent()<<<, user));
   ```
   ).
   (I know it's the fault of the original design but we are using "parent" for 2 different concepts here which is fairly confusing.)
   
   Maybe calling the two `MatchEnriching` instance as for example `componentMatchEnricher` and `scopeMatchEnricher/currentGroupMatchEnricher` would help.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] mcgilman commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
mcgilman commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392524417
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
 ##########
 @@ -2118,6 +2118,13 @@
             return nfCanvas.getParentGroupId();
         },
 
+        /**
+         * Get the group id.
+         */
+        getGroupId: function () {
 
 Review comment:
   This function already exists in `nfCanvasUtils`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392328379
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/PortScheduledStateMatcher.java
 ##########
 @@ -0,0 +1,52 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public class PortScheduledStateMatcher implements AttributeMatcher<Port> {
+    private static final String SEARCH_TERM_DISABLED = "disabled";
+    private static final String SEARCH_TERM_INVALID = "invalid";
+    private static final String SEARCH_TERM_RUNNING = "running";
+    private static final String SEARCH_TERM_STOPPED = "stopped";
+
+    private static final String MATCH_PREFIX = "Run status: ";
+    private static final String MATCH_DISABLED = "Disabled";
+    private static final String MATCH_INVALID = "Invalid";
+    private static final String MATCH_RUNNING = "Running";
+    private static final String MATCH_STOPPED = "Stopped";
+
+    @Override
+    public void match(final Port component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (ScheduledState.DISABLED.equals(component.getScheduledState()) && StringUtils.containsIgnoreCase(SEARCH_TERM_DISABLED, searchTerm)) {
 
 Review comment:
   This if-else branching is not logically the same as the original.
   For example in your case the second branch might get evaluated if the scheduled state is `DISABLED` but there's no `"disabled"` (or something like that) in the search term.
   
   With the original logic it was not the case - If the component was `DISABLED`, the other cases did not get evaluated period.
   
   Doesn't seem to be too significant but maybe (not sure) _disabled_ components _can_ be _invalid_ (and/or _stopped_) at the same time, and you probably don't want those to come up when you are looking for invalid/stopped components in general.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393202870
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/SearchQuery.java
 ##########
 @@ -0,0 +1,72 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+/**
+ * Represents the data set the search query executes based on.
+ */
+public interface SearchQuery {
+
+    /**
+     * The part of the query string not containing metadata (filters).
+     *
+     * @return The query string used for executing the search.
+     */
+    String getTerm();
+
+    /**
+     * Returns if the query contains a given filter (regardless the value).
 
 Review comment:
   `Returns whether...` may be better. (`Returns if...` suggests the method returns only if the conditions are met.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393114289
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultErnicher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, groupResultErnicher, results.getProcessGroupResults());
 
 Review comment:
   It's correct and fixed. Even if we do want to search in the components contained by the process group, the group itself might not be subject of the search.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392392558
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,633 +16,214 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches term in the controller beginning from a given process group.
-     *
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultErnicher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, groupResultErnicher, results.getProcessGroupResults());
 
 Review comment:
   I think in the original logic there was check to skip the group if it has no parent (i.e. the root group).
   We probably want to keep that. (Otherwise the enricher will throw NPE.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401830178
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,688 +16,222 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
 import org.apache.nifi.controller.service.ControllerServiceNode;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
-import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.Optional;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
+    private ComponentMatcher<ControllerServiceNode> matcherForControllerServiceNode;
 
     /**
-     * Searches term in the controller beginning from a given process group.
+     * Searches all parameter contexts and parameters.
      *
+     * @param searchQuery Details of the search
      * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
      */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
+        } else {
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
+    }
 
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final ComponentSearchResultEnricher resultEnricher = resultEnricherFactory.getComponentResultEnricher(scope, user);
+        final ComponentSearchResultEnricher groupResultEnricher = resultEnricherFactory.getProcessGroupResultEnricher(scope, user);
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            if (scope.getParent() != null) {
+                searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, groupResultEnricher, results.getProcessGroupResults());
             }
-        }
 
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, resultEnricher, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, resultEnricher, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, resultEnricher, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, resultEnricher, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, resultEnricher, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, resultEnricher, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, resultEnricher, results.getLabelResults());
+            searchComponentType(scope.getControllerServices(false), user, searchQuery, matcherForControllerServiceNode, resultEnricher, results.getControllerServiceNodeResults());
         }
 
-        for (final ControllerServiceNode controllerServiceNode : group.getControllerServices(false)) {
-            if (controllerServiceNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, controllerServiceNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getControllerServiceNodeResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    /**
-     * Searches controller service for the given search term
-     *
-     * @param search                the search term
-     * @param controllerServiceNode a group controller service node
-     */
-    private ComponentSearchResultDTO search(final String search, final ControllerServiceNode controllerServiceNode) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(search, controllerServiceNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(search, controllerServiceNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(search, controllerServiceNode.getName(), "Name", matches);
-        addIfAppropriate(search, controllerServiceNode.getComments(), "Comments", matches);
-
-        // search property values
-        controllerServiceNode.getRawPropertyValues().forEach((property, propertyValue) -> {
-            addIfAppropriate(search, property.getName(), "Property Name", matches);
-            addIfAppropriate(search, property.getDescription(), "Property Description", matches);
-
-            // never include sensitive properties in search results
-            if (property.isSensitive()) {
-                return;
-            }
-
-            if (propertyValue != null) {
-                addIfAppropriate(search, propertyValue, "Property Value", matches);
-            } else {
-                addIfAppropriate(search, property.getDefaultValue(), "Property Value", matches);
-            }
-        });
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(controllerServiceNode.getIdentifier());
-        dto.setName(controllerServiceNode.getName());
-        dto.setMatches(matches);
-        return dto;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return !searchQuery.hasFilter(FILTER_NAME_GROUP) || eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP));
     }
 
     /**
-     * Searches all parameter contexts and parameters
+     * Check is the group is eligible for the filter value. It might be eligible based on name or id.
      *
-     * @param results Search results
-     * @param search  The search term
+     * @param scope The subject process group.
+     * @param filterValue The value to match against.
+     *
+     * @return True in case the scope process group or any parent is matching. A group is matching when it's name or it's id contains the filter value.
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<ProcessGroup> lineage = getLineage(scope);
 
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
+        for (final ProcessGroup group : lineage) {
+            if (StringUtils.containsIgnoreCase(group.getName(), filterValue) || StringUtils.containsIgnoreCase(group.getIdentifier(), filterValue)) {
+                return true;
             }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
         }
 
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
+    private List<ProcessGroup> getLineage(final ProcessGroup group) {
+        final LinkedList<ProcessGroup> result = new LinkedList<>();
 
 Review comment:
   Just an idea but it would be more straightforward for me in this way:
   
   ```
           final LinkedList<ProcessGroup> result = new LinkedList<>();
           
           ProcessGroup current = group;
           while (current != null) {
               result.addLast(current);
               current = current.getParent();
           }
   
           return result;
   ```
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400923296
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
 
 Review comment:
   ```suggestion
   *back pressure*: Adds Connections to the result list that are applying back pressure at the time.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393204945
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/SearchQuery.java
 ##########
 @@ -0,0 +1,72 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+/**
+ * Represents the data set the search query executes based on.
+ */
+public interface SearchQuery {
+
+    /**
+     * The part of the query string not containing metadata (filters).
+     *
+     * @return The query string used for executing the search.
+     */
+    String getTerm();
+
+    /**
+     * Returns if the query contains a given filter (regardless the value).
+     *
+     * @param filterName The name of the filter.
+     *
+     * @return True if the query contains the filter, false otherwise.
+     */
+    boolean hasFilter(String filterName);
+
+    /**
+     * Returns the value of the query filter. Should be checked with {@link #hasFilter} beforehand!
+     *
+     * @param filterName The name of the filter.
+     *
+     * @return The value of the filter if exists, otherwise it's null.
+     */
+    String getFilter(String filterName);
+
+    /**
+     * Returns the user who executes the query.
+     *
+     * @return Requesting user.
+     */
+    NiFiUser getUser();
+
+    /**
+     * Reference to the flow's root process group.
+     *
+     * @return Root group of the flow.
+     */
+    ProcessGroup getRootGroup();
+
+    /**
+     * Reference to the process group was active for the user during requesting the search. This might be the same as the root group.
 
 Review comment:
   Reference to the process group _which_ was active... 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r397054353
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceIntegrationTest.java
 ##########
 @@ -0,0 +1,598 @@
+/*
+ * 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.web.controller;
+
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.validation.ValidationStatus;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.controller.queue.FlowFileQueue;
+import org.apache.nifi.flowfile.FlowFilePrioritizer;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.ComponentVariableRegistry;
+import org.apache.nifi.registry.VariableDescriptor;
+import org.apache.nifi.scheduling.ExecutionNode;
+import org.apache.nifi.scheduling.SchedulingStrategy;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.junit.Assert;
+import org.junit.Test;
+import org.mockito.Mockito;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.nifi.web.controller.ComponentMockUtil.getBasicRelationships;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getChildProcessGroup;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getConnection;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getFunnel;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getPort;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getProcessorNode;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getPublicPort;
+import static org.apache.nifi.web.controller.ComponentMockUtil.getRemoteProcessGroup;
+
+public class ControllerSearchServiceIntegrationTest extends AbstractControllerSearchIntegrationTest {
+
+    @Test
+    public void testSearchBasedOnBasicAttributes() {
+        // given
+        givenRootProcessGroup()
+                .withProcessor(getProcessorNode("processor1", "name1", AUTHORIZED))
+                .withProcessor(getProcessorNode("processor2", "NAME2", AUTHORIZED))
+                .withProcessor(getProcessorNode("processor3", "name3", NOT_AUTHORIZED))
+                .withProcessor(getProcessorNode("processor4", "other", AUTHORIZED))
+                .withProcessor(getProcessorNode("processor5", "something", "The name of the processor is something",
+                        Optional.of("versionId"), SchedulingStrategy.TIMER_DRIVEN, ExecutionNode.ALL, ScheduledState.RUNNING, ValidationStatus.VALID,
+                        new HashSet<>(),"Processor", Mockito.mock(Processor.class), new HashMap<>(), AUTHORIZED));
+
+        givenProcessGroup(getChildProcessGroup("childId", "child", "", getProcessGroup(ROOT_PROCESSOR_GROUP_ID), AUTHORIZED, NOT_UNDER_VERSION_CONTROL))
+                .withInputPort(getPort("port1", "name4", "comment consisting name", ScheduledState.RUNNING, true, AUTHORIZED))
+                .withOutputPort(getPort("port2", "TheName5", "comment", ScheduledState.RUNNING, true, AUTHORIZED))
+                .withFunnel(getFunnel("hasNoName1", Optional.empty(), AUTHORIZED))
+                .withFunnel(getFunnel("hasNoName2", Optional.empty(), NOT_AUTHORIZED));
+
+        // when
+        whenExecuteSearch("name");
+
+        // then
+        thenResultConsists()
+                .ofProcessor(getSimpleResultFromRoot("processor1", "name1", "Name: name1"))
+                .ofProcessor(getSimpleResultFromRoot("processor2", "NAME2", "Name: NAME2"))
+                .ofProcessor(getSimpleResultFromRoot("processor5", "something", "Comments: The name of the processor is something"))
+                .ofInputPort(getSimpleResult("port1", "name4", "childId", "childId", "child", "Name: name4", "Comments: comment consisting name"))
+                .ofOutputPort(getSimpleResult("port2", "TheName5", "childId", "childId", "child", "Name: TheName5"))
+                .ofFunnel(getSimpleResult("hasNoName1", null, "childId", "childId", "child", "Id: hasNoName1"))
+                .validate(results);
+    }
+
+    @Test
+    public void testSearchBasedOnScheduling() {
+        // given
+        givenRootProcessGroup()
+                .withProcessor(getProcessorNode("processor1", "processor1name", SchedulingStrategy.EVENT_DRIVEN, ExecutionNode.ALL, ScheduledState.RUNNING, ValidationStatus.VALID, AUTHORIZED))
+                .withProcessor(getProcessorNode("processor2", "processor2name", SchedulingStrategy.EVENT_DRIVEN, ExecutionNode.ALL, ScheduledState.DISABLED, ValidationStatus.INVALID, AUTHORIZED))
+                .withProcessor(getProcessorNode("processor3", "processor3name", SchedulingStrategy.EVENT_DRIVEN, ExecutionNode.ALL, ScheduledState.RUNNING, ValidationStatus.VALID, NOT_AUTHORIZED))
+                .withProcessor(getProcessorNode("processor4", "processor4name", SchedulingStrategy.TIMER_DRIVEN, ExecutionNode.ALL, ScheduledState.STOPPED, ValidationStatus.VALID, AUTHORIZED))
+                .withProcessor(getProcessorNode("processor5", "eventHandlerProcessor", SchedulingStrategy.CRON_DRIVEN, ExecutionNode.PRIMARY, ScheduledState.RUNNING, ValidationStatus.VALID,
+                        AUTHORIZED));
+
+        // when
+        whenExecuteSearch("event");
+
+        // then
+        thenResultConsists()
+                .ofProcessor(getSimpleResultFromRoot("processor1", "processor1name", "Scheduling strategy: Event driven"))
+                .ofProcessor(getSimpleResultFromRoot("processor2", "processor2name", "Scheduling strategy: Event driven"))
+                .ofProcessor(getSimpleResultFromRoot("processor5", "eventHandlerProcessor", "Name: eventHandlerProcessor"));
+
+
+        // when
+        whenExecuteSearch("timer");
+
+        // then
+        thenResultConsists()
+                .ofProcessor(getSimpleResultFromRoot("processor4", "processor4name", "Scheduling strategy: Timer driven"));
+
+
+        // when
+        whenExecuteSearch("primary");
+
+        // then
+        thenResultConsists()
 
 Review comment:
   No `.validate(results)` is called
   Expected value should be `"Execution node: primary"`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391167931
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/ExpirationAttributeMatcher.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class ExpirationAttributeMatcher implements AttributeMatcher<Connection> {
+    private static final String MATCH_PREFIX = "FlowFile expiration: ";
+    private static final Set<String> KEYWORDS = new HashSet<>(Arrays.asList( //
+            "expires", //
+            "expiration"));
+
+    @Override
+    public void match(final Connection component, final SearchQuery query, final List<String> matches) {
+        if (containsKeyword(query)) {
+            final int expirationMillis = component.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
+
+            if (expirationMillis > 0) {
+                matches.add(MATCH_PREFIX + component.getFlowFileQueue().getFlowFileExpiration());
+            }
+        }
+    }
+
+    private boolean containsKeyword(final SearchQuery query) {
+        for(final String keyword : KEYWORDS) {
 
 Review comment:
   Minor suggestion:
   We already started using the stream API. How about
   ```java
   return KEYWORDS.stream().anyMatch(keyword -> StringUtils.containsIgnoreCase(keyword, query.getTerm()));
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401210406
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+Users can use pre-defined (case-insensitive) keywords in the search box that will check certain conditions (instead of trying to match labels and values etc.). Keywords can be used with filters (see below) but not with other search terms (otherwise they won't be treated as keywords) and only one keyword can be used at a time. Note however that keywords will also be treated as general search terms at the same time. The supported keywords are the following:
+
+*back pressure*: Adds Connections to the result list that are applying back pressure at the time.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds Connections to the result list that contain expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds Processors to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list that are not transmitting data at the time.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors to the result list that are set to run on the primary node only (whether if the Processor is currently running or not).
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds Processors to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list that are transmitting data at the time.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list that are validating at the time.
+
+=== Filters
+
+Filters can be added to the search box as key-value pairs where the keys are predefined and check certain conditions based on the given value. The syntax is "key:value". Filters can be used together with other search terms and multiple filters can be used. The only constraint is that the search must start with the filters. Unknown filters or known filters with unknown values are ignored. If the same filter key appears multiple times, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "here". The usage of this filter looks like "scope:here". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. Search will be restricted to groups (and their components - including subgroups and their components) the names or ids of which match the filter value. If no group matches the filter the result list will be empty.
 
 Review comment:
   Minor: missing comma in the last sentence: "If no group matches the filter, the result list..."

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401759140
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
 
 Review comment:
   I think it would be better to move this section to a separate chapter and not to dive into the details here, at the beginning of the user guide.
   I would also add an intro paragraph with some general info about the search capability.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391150825
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
 ##########
 @@ -75,6 +75,128 @@
         <property name="dtoFactory" ref="dtoFactory" />
     </bean>
 
+    <!-- search functionality -->
+    <bean id="searchQueryParser" class="org.apache.nifi.web.search.query.RegexSearchQueryParser" />
+
+    <bean id="attributeMatcher" class="org.apache.nifi.web.search.attributematchers.BasicAttributeMatcher" />
+    <bean id="limitedAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.BasicAttributeMatcher">
+        <constructor-arg value="false" />
+    </bean>
+    <bean id="schedulingMatcher" class="org.apache.nifi.web.search.attributematchers.SchedulingMatcher" />
+    <bean id="executionMatcher" class="org.apache.nifi.web.search.attributematchers.ExecutionMatcher" />
+    <bean id="scheduledStateMatcher" class="org.apache.nifi.web.search.attributematchers.ScheduledStateMatcher" />
+    <bean id="relationshipMatcher" class="org.apache.nifi.web.search.attributematchers.RelationshipMatcher" />
+    <bean id="processorMetadataMatcher" class="org.apache.nifi.web.search.attributematchers.ProcessorMetadataMatcher" />
+    <bean id="propertyMatcher" class="org.apache.nifi.web.search.attributematchers.PropertyMatcher" />
+    <bean id="searchableMatcher" class="org.apache.nifi.web.search.attributematchers.SearchableMatcher">
+        <property name="variableRegistry" ref="variableRegistry" />
 
 Review comment:
   Don't we need to inject `flowController` as well?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391591983
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/ExpirationAttributeMatcher.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+
+public class ExpirationAttributeMatcher implements AttributeMatcher<Connection> {
+    private static final String MATCH_PREFIX = "FlowFile expiration: ";
+    private static final Set<String> KEYWORDS = new HashSet<>(Arrays.asList( //
+            "expires", //
+            "expiration"));
+
+    @Override
+    public void match(final Connection component, final SearchQuery query, final List<String> matches) {
+        if (containsKeyword(query)) {
+            final int expirationMillis = component.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
+
+            if (expirationMillis > 0) {
+                matches.add(MATCH_PREFIX + component.getFlowFileQueue().getFlowFileExpiration());
+            }
+        }
+    }
+
+    private boolean containsKeyword(final SearchQuery query) {
+        for(final String keyword : KEYWORDS) {
 
 Review comment:
   Looks better, I applicate it for every case like this in the matchers.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396091942
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
 ##########
 @@ -197,9 +197,10 @@
      * Searches the controller for the specified query string.
      *
      * @param query query
+     * @param activeGroupId the id of the group currently visible in the editor
 
 Review comment:
   I have nothing against it but it is confusing anyway: what I select might be one which I select with mouse pointer, but I am not visiting it's actual content (just the box representation)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391077893
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/resources/nifi-web-api-context.xml
 ##########
 @@ -75,6 +75,128 @@
         <property name="dtoFactory" ref="dtoFactory" />
     </bean>
 
+    <!-- search functionality -->
+    <bean id="searchQueryParser" class="org.apache.nifi.web.search.query.RegexSearchQueryParser" />
+
+    <bean id="attributeMatcher" class="org.apache.nifi.web.search.attributematchers.BasicAttributeMatcher" />
+    <bean id="limitedAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.BasicAttributeMatcher">
+        <constructor-arg value="false" />
+    </bean>
+    <bean id="schedulingMatcher" class="org.apache.nifi.web.search.attributematchers.SchedulingMatcher" />
+    <bean id="executionMatcher" class="org.apache.nifi.web.search.attributematchers.ExecutionMatcher" />
+    <bean id="scheduledStateMatcher" class="org.apache.nifi.web.search.attributematchers.ScheduledStateMatcher" />
+    <bean id="relationshipMatcher" class="org.apache.nifi.web.search.attributematchers.RelationshipMatcher" />
+    <bean id="processorMetadataMatcher" class="org.apache.nifi.web.search.attributematchers.ProcessorMetadataMatcher" />
+    <bean id="propertyMatcher" class="org.apache.nifi.web.search.attributematchers.PropertyMatcher" />
+    <bean id="searchableMatcher" class="org.apache.nifi.web.search.attributematchers.SearchableMatcher">
+        <property name="variableRegistry" ref="variableRegistry" />
+    </bean>
+    <bean id="processGroupAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ProcessGroupAttributeMatcher" />
+    <bean id="componentVariableRegistryAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ComponentVariableRegistryAttributeMatcher" />
+    <bean id="connectionAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ConnectionAttributeMatcher" />
+    <bean id="prioritiesAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.PrioritiesAttributeMatcher" />
+    <bean id="expirationAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ExpirationAttributeMatcher" />
+    <bean id="backPressureAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.BackPressureAttributeMatcher" />
+    <bean id="connectivityAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ConnectivityAttributeMatcher" />
+    <bean id="remoteProcessGroupAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.RemoteProcessGroupAttributeMatcher" />
+    <bean id="targetUriMatcher" class="org.apache.nifi.web.search.attributematchers.TargetUriMatcher" />
+    <bean id="transmissionStatusMatcher" class="org.apache.nifi.web.search.attributematchers.TransmissionStatusMatcher" />
+    <bean id="portScheduledStateMatcher" class="org.apache.nifi.web.search.attributematchers.PortScheduledStateMatcher" />
+    <bean id="publicPortMatcher" class="org.apache.nifi.web.search.attributematchers.PublicPortMatcher" />
+    <bean id="parameterContextAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.ParameterContextAttributeMatcher" />
+    <bean id="parameterMatcher" class="org.apache.nifi.web.search.attributematchers.ParameterMatcher" />
+    <bean id="labelAttributeMatcher" class="org.apache.nifi.web.search.attributematchers.LabelAttributeMatcher" />
+
+    <bean id="componentMatcherFactory" class="org.apache.nifi.web.search.ComponentMatcherFactory" />
+
+    <bean id="matcherForProcessor" factory-bean="componentMatcherFactory" factory-method="getInstanceForConnectable">
+        <constructor-arg>
+            <util:list>
+                <ref bean="attributeMatcher" />
+                <ref bean="schedulingMatcher" />
+                <ref bean="executionMatcher" />
+                <ref bean="scheduledStateMatcher" />
+                <ref bean="relationshipMatcher" />
+                <ref bean="processorMetadataMatcher" />
+                <ref bean="propertyMatcher" />
+                <ref bean="searchableMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForProcessGroup" factory-bean="componentMatcherFactory" factory-method="getInstanceForProcessGroup">
+        <constructor-arg>
+            <util:list>
+                <ref bean="processGroupAttributeMatcher" />
+                <ref bean="componentVariableRegistryAttributeMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForConnection" factory-bean="componentMatcherFactory" factory-method="getInstanceForConnection">
+        <constructor-arg>
+            <util:list>
+                <ref bean="connectionAttributeMatcher" />
+                <ref bean="relationshipMatcher" />
+                <ref bean="prioritiesAttributeMatcher" />
+                <ref bean="expirationAttributeMatcher" />
+                <ref bean="backPressureAttributeMatcher" />
+                <ref bean="connectivityAttributeMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForRemoteProcessGroup" factory-bean="componentMatcherFactory" factory-method="getInstanceForRemoteProcessGroup">
+        <constructor-arg>
+            <util:list>
+                <ref bean="remoteProcessGroupAttributeMatcher" />
+                <ref bean="targetUriMatcher" />
+                <ref bean="transmissionStatusMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForPort" factory-bean="componentMatcherFactory" factory-method="getInstanceForConnectable">
+        <constructor-arg>
+            <util:list>
+                <ref bean="attributeMatcher" />
+                <ref bean="portScheduledStateMatcher" />
+                <ref bean="publicPortMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForFunnel" factory-bean="componentMatcherFactory" factory-method="getInstanceForConnectable">
+        <constructor-arg>
+            <util:list>
+                <ref bean="limitedAttributeMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForParameterContext" factory-bean="componentMatcherFactory" factory-method="getInstanceForParameterContext">
+        <constructor-arg>
+            <util:list>
+                <ref bean="parameterContextAttributeMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForParameter" factory-bean="componentMatcherFactory" factory-method="getInstanceForParameter">
+        <constructor-arg>
+            <util:list>
+                <ref bean="parameterMatcher" />
+            </util:list>
+        </constructor-arg>
+    </bean>
+
+    <bean id="matcherForLabel" factory-bean="componentMatcherFactory" factory-method="getInstanceForConnectable">
 
 Review comment:
   `Labels` are not `Connectables`!

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400921543
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
 
 Review comment:
   I think these document sections should be worded as simply as possible.
   `literal` for example in the sense it is used here is a computer science specific expression and meant to be used for fixed values in the _source code_.
   In general it should read naturally for basically any user, here's a suggestion:
   ```
   Users can use pre-defined (case-insensitive) keywords in the search box that will check certain conditions (instead of trying to match labels and values etc.).
   Keywords can be used with filters (see below) but not with other search terms (otherwise they won't be treated as keywords) and only one keyword can be used at a time.
   Note however that keywords will also be treated as general search terms at the same time.
   The supported keywords are the following:
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390370433
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
 
 Review comment:
   Regarding the `//` at the end of the lines - while the automatic formatters might not do a perfect job, I think we still should avoid these kinds of "hacky" approaches.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390475982
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/MatchEnriching.java
 ##########
 @@ -0,0 +1,56 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+public class MatchEnriching implements Function<ComponentSearchResultDTO, ComponentSearchResultDTO> {
+    private final Optional<String> groupIdentifier;
+    private final Optional<SearchResultGroupDTO> parentGroup;
+    private final Optional<SearchResultGroupDTO> versionedGroup;
+
+    public MatchEnriching(final String groupIdentifier, final SearchResultGroupDTO parentGroup, final SearchResultGroupDTO versionedGroup) {
+        this(Optional.ofNullable(groupIdentifier), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+    }
+
+    public MatchEnriching(final Optional<String> groupIdentifier, final Optional<SearchResultGroupDTO> parentGroup, final Optional<SearchResultGroupDTO> versionedGroup) {
+        this.groupIdentifier = groupIdentifier;
+        this.parentGroup = parentGroup;
+        this.versionedGroup = versionedGroup;
+    }
+
+    @Override
+    public ComponentSearchResultDTO apply(final ComponentSearchResultDTO match) {
+
 
 Review comment:
   Minor suggestion: Maybe a bit simpler:
   ```java
           groupIdentifier.ifPresent(match::setGroupId);
           parentGroup.ifPresent(match::setParentGroup);
           versionedGroup.ifPresent(match::setVersionedGroup);
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400926142
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
 
 Review comment:
   ```suggestion
   *not transmitting*: Adds Remote Process Groups to the result list that are not transmitting data at the time.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396093495
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenContentOfTheFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        int numberOfCheckedGroups = 0;
+
+        for (final String processGroup : searchedProcessGroups) {
+            // Checking on funnels is arbitrary, any given component we expect to be searched would be a good candidate
+            Mockito.verify(processGroups.get(processGroup), Mockito.times(1)).getFunnels();
+            numberOfCheckedGroups++;
+        }
+
+        Assert.assertEquals(searchedProcessGroups.size(), numberOfCheckedGroups);
 
 Review comment:
   It's not useful anymore, I do remove

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400948943
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
+
+*group*: This filter narrows the scope of the search based on the provided Process Group name or id. If provided, the only groups will be searched are the ones containing the filter value in their name or id. This is transitive, so if any parent Process Group is fit, than the contained Process Groups are subject of the search. In case on non-existing group, the filter will be used but the result list will be empty.
+
+*properties*: In case of valid value, this filter excludes the findings would be added based on property. The valid values are: "_no_", "_none_", "_false_", "_exclude_" and "_0_". Every other value is invalid, resulting that the search will ignore the filter.
+
+=== Search examples
+
+==== Example 1
+
+`group:myGroup processor1`
+
+The search query will result a list of components are containing `processor1` in their relevant attributes. The search will be executed under Process Groups (directly or via contained Process Groups) containing the string "myGroup" in their name or id.
 
 Review comment:
   ```suggestion
   The search will be executed under Process Groups (directly or via contained Process Groups) containing the string "myGroup" in their name or id.
   The result will contain components that match for "processor1".
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392888593
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/GeneralComponentSearchResultEnricher.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+
+public class GeneralComponentSearchResultEnricher extends AbstractComponentSearchResultEnricher {
+    public GeneralComponentSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        super(processGroup, user, authorizer);
+    }
+
+    @Override
+    public ComponentSearchResultDTO enrich(final ComponentSearchResultDTO input) {
+        input.setGroupId(processGroup.getIdentifier());
+        input.setParentGroup(buildResultGroup(processGroup.getParent(), user));
 
 Review comment:
   Yes you are correct, we should. (Corrected already)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392221336
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -147,8 +148,8 @@
     // properties
     private NiFiProperties properties;
     private DtoFactory dtoFactory;
-    private VariableRegistry variableRegistry;
     private ControllerSearchService controllerSearchService;
+    private SearchQueryParser searchQueryParser;
 
 Review comment:
   Could you please follow the same order of properties as in `nifi-web-api-context.xml`?
   First `searchQueryParser`, then `controllerSearchService`.
   The same for the setters. (or fix the order in the spring xml)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393244209
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
 
 Review comment:
   We could remove these and add
   ```java
           children.forEach(child -> Mockito.when(child.getParent()).thenReturn(result));
   ```
   in `givenProcessGroup` instead.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393217416
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/NiFiServiceFacade.java
 ##########
 @@ -197,9 +197,10 @@
      * Searches the controller for the specified query string.
      *
      * @param query query
+     * @param activeGroupId the id of the group currently visible in the editor
 
 Review comment:
   `currently selected` instead of `currently visible` may be better. (On the canvas you might _see_ multiple process groups in the currently _selected_ one.)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400924050
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
 
 Review comment:
   ```suggestion
   *expiration*: Adds Connections to the result list that contain expired Flow Files.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393119573
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/ProcessGroupSearchResultEnricher.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+
+public class ProcessGroupSearchResultEnricher extends AbstractComponentSearchResultEnricher {
+
+    public ProcessGroupSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        super(processGroup, user, authorizer);
+    }
+
+    @Override
+    public ComponentSearchResultDTO enrich(final ComponentSearchResultDTO input) {
+        input.setParentGroup(buildResultGroup(processGroup.getParent(), user));
 
 Review comment:
   Yes you are right. It did mislead me that the other components usually has this outside the search method, just above `setParentGroup` and `setVersionedGroup`. in case of process group it is within the search method. Fixed.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400928982
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
 
 Review comment:
   ```suggestion
   *timer*: Adds Processors to the result list where the Scheduling Strategy is "Event Timer".
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393247334
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
 
 Review comment:
   We could remove these (and the same in `givenSingleProcessGroupIsSetUp`) and add
   ```java
           processGroups = new HashMap<>();
   ```
   in `setUp` as well as add
   ```java
           processGroups.put(identifier, result);
   ```
   in `givenProcessGroup` instead.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] mcgilman commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
mcgilman commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392525507
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -1614,15 +1615,20 @@ private void setComponentDetails(final ProvenanceEventDTO dto) {
     /**
      * Searches this controller for the specified term.
      *
-     * @param search search
+     * @param searchLiteral search
+     * @param activeGroupId the identifier of the currently visited group
      * @return result
      */
-    public SearchResultsDTO search(final String search) {
+    public SearchResultsDTO search(final String searchLiteral, final String activeGroupId) {
         final ProcessGroup rootGroup = getRootGroup();
+        final ProcessGroup activeGroup =  flowController.getFlowManager().getGroup(activeGroupId);
 
 Review comment:
   `getGroup` here requires non null. While the update to our primary client (the UI) has been updated to always include the active group other clients (like scripts or CLIs) may not be including the new parameter. If the `activeGroupId` is null I would recommend using the root group which should maintain existing functionality.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392393361
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/ProcessGroupSearchResultEnricher.java
 ##########
 @@ -0,0 +1,36 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+
+public class ProcessGroupSearchResultEnricher extends AbstractComponentSearchResultEnricher {
+
+    public ProcessGroupSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        super(processGroup, user, authorizer);
+    }
+
+    @Override
+    public ComponentSearchResultDTO enrich(final ComponentSearchResultDTO input) {
+        input.setParentGroup(buildResultGroup(processGroup.getParent(), user));
 
 Review comment:
   I think the original had this which could be placed in this enricher: `result.setGroupId(parent.getIdentifier();`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390406702
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/RegexSearchQueryParser.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+import javax.annotation.Nonnull;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class RegexSearchQueryParser implements SearchQueryParser {
+    private static final String REGEX = "(?<filter>(([\\w]+\\:[\\w]+[\\s]+)*)(([\\w]+\\:[\\w]+){0,1}))((<?<other>[\\w]+\\:[\\w]+)|(?<term>.*))";
 
 Review comment:
   It seems this pattern can be simplified.
   For example this pattern seems to cover the same cases (at least passes all the tests):
   ```java
   private static final String REGEX = "(?<filter>(\\w+:\\w+\\s+)*(\\w+:\\w+)?)(?<term>.*)";
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391145472
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/PropertyMatcher.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class PropertyMatcher implements AttributeMatcher<ProcessorNode> {
+    private static final String LABEL_NAME = "Property name";
+    private static final String LABEL_VALUE = "Property value";
+    private static final String LABEL_DESCRIPTION = "Property description";
+
+    private final static String FILTER_NAME_PROPERTIES = "properties";
+    private final static Set<String> FILTER_VALUES_PROPERTIES_EXCLUSION = new HashSet<>(Arrays.asList("no", "none", "false", "exclude", "0"));
+
+    @Override
+    public void match(final ProcessorNode component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (!propertiesAreFilteredOut(query)) {
+            for (final Map.Entry<PropertyDescriptor, String> entry : component.getRawPropertyValues().entrySet()) {
+                final PropertyDescriptor descriptor = entry.getKey();
+                addIfMatching(searchTerm, descriptor.getName(), LABEL_NAME, matches);
+                addIfMatching(searchTerm, descriptor.getDescription(), LABEL_DESCRIPTION, matches);
+
+                // never include sensitive properties values in search results
+                if (!descriptor.isSensitive()) {
+                    final String value = (entry.getValue() == null)
 
 Review comment:
   I don't think that this expression with the ternary operator is more readable than the original.
   If anything, I'd go with `Optional`, like this:
   ```java
   final String value = Optional.ofNullable(entry.getValue()).orElse(descriptor.getDefaultValue());
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401919427
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/StandardNiFiServiceFacade.java
 ##########
 @@ -342,7 +342,7 @@
 /**
  * Implementation of NiFiServiceFacade that performs revision checking.
  */
-public class StandardNiFiServiceFacade implements NiFiServiceFacade {
+public class  StandardNiFiServiceFacade implements NiFiServiceFacade {
 
 Review comment:
   Seems to be an unintentional change.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396090564
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -147,8 +148,8 @@
     // properties
     private NiFiProperties properties;
     private DtoFactory dtoFactory;
-    private VariableRegistry variableRegistry;
     private ControllerSearchService controllerSearchService;
+    private SearchQueryParser searchQueryParser;
 
 Review comment:
   Sure, I arranged them into the same order

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396093080
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
 
 Review comment:
   I wanted to ensure that there is no further call. This seems even more accurate for this intention.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390934339
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/BasicAttributeMatcher.java
 ##########
 @@ -0,0 +1,55 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.nifi.connectable.Connectable;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class BasicAttributeMatcher<T extends Connectable> implements AttributeMatcher<T> {
 
 Review comment:
   I completely agree and I hesitated a lot. The reason I went this way is the following: I wanted to reduce code duplication around search where it is reasonable and originally I was hoping for one class with this behaviour. But as the `getName` and `getIdentifier` methods in the different components are not inherited from the same parent it ended with multiple classes with almost the same behaviour. It looked a good idea to not add one more, but on second though... It might also makes sense to follow an approach similar to the `AttributeBasedComponentMatcher`. It comes with a somewhat more complex instantiation but is more flexible. What is your opinion?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390932356
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/query/RegexSearchQueryParser.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.web.search.query;
+
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+
+import javax.annotation.Nonnull;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class RegexSearchQueryParser implements SearchQueryParser {
+    private static final String REGEX = "(?<filter>(([\\w]+\\:[\\w]+[\\s]+)*)(([\\w]+\\:[\\w]+){0,1}))((<?<other>[\\w]+\\:[\\w]+)|(?<term>.*))";
 
 Review comment:
   I take a look on this. I was experimenting a while until I found a pattern works fine for everything but I can imagine there are simpler ones.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390859318
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -147,8 +148,8 @@
     // properties
     private NiFiProperties properties;
     private DtoFactory dtoFactory;
-    private VariableRegistry variableRegistry;
     private ControllerSearchService controllerSearchService;
+    private SearchQueryParser searchQueryParser;
 
 Review comment:
   It is a user input parser, and based on that I think it is belongs more to the controller side than the service side. I think it is more clear from the perspective of boundaries if the service is not responsible to parse the user input string but working with a request object. Also from the practical side: as the "search" and the "searchParameters" are called separately from the controller facade and they both need the user input, it looks unnecessary to parse it twice within the search methods. (This separation that they are distinct calls is a decision from the original code which I did not intend to change)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390355100
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,150 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
 
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
-
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
-            }
-        }
-
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? getAncestors(scope).contains(searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getAncestors(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
 
 Review comment:
   Is there a reason to collect the contexts?
   The stream processing could fluently continue.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390485055
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/MatchEnriching.java
 ##########
 @@ -0,0 +1,56 @@
+/*
+ * 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.web.search;
+
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
+import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
+
+import java.util.Optional;
+import java.util.function.Function;
+
+public class MatchEnriching implements Function<ComponentSearchResultDTO, ComponentSearchResultDTO> {
 
 Review comment:
   This class is perfectly fine if it doesn't implement `Function`.
   The advantage would be that the `apply` method would be freed up and
   1. could be renamed (to `enrich` for example)
   1. would be easier to find in a call hierarchy (which really helps with finding out what it is used for)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390537947
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
 
 Review comment:
   The same `parentGroup` and `versionedGroup` is used for all match results. It's probably fine, but I feel uncomfortable about it because `SearchResultGroupDTO` is _not_ immutable.
   I think we should _not_ try to save the creation of those objects unless we can responsibly change it to be immutable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393009316
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -1614,15 +1615,20 @@ private void setComponentDetails(final ProvenanceEventDTO dto) {
     /**
      * Searches this controller for the specified term.
      *
-     * @param search search
+     * @param searchLiteral search
 
 Review comment:
   It sounds more describing, thanks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400925645
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
 
 Review comment:
   ```suggestion
   *event*: Adds Processors to the result list where the Scheduling Strategy is "Event Driven".
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r400965152
 
 

 ##########
 File path: nifi-docs/src/main/asciidoc/user-guide.adoc
 ##########
 @@ -167,6 +167,74 @@ contains options that allow you to manipulate existing components on the canvas:
 
 image::global-menu.png[NiFi Global Menu]
 
+=== Keywords in searching
+
+NiFi UI's search functionality supports a number of "keywords". These are literals used to execute search based on non-textual information. A keyword is used only in case of the search term consists of a section of the keyword or matches with it. In case the keyword is only the subset of the search term, the keyword is not picked for usage. The keywords are case-insensitive. The supported keywords are the following:
+
+*back pressure*: Adds the Connections to the result list which are applying back pressure in the time of search.
+
+*disabled*: Adds disabled Ports and Processors to the result list.
+
+*expiration*: Adds the Connections to the result list in which there are expired Flow Files.
+
+*expires*: See "expiration".
+
+*event*: Adds every Processor to the result list where the Scheduling Strategy is "Event Driven".
+
+*invalid*: Adds Ports and Processors to the result list where the component is invalid.
+
+*not transmitting*: Adds Remote Process Groups to the result list which are not transmitting data at the time of searching.
+
+*pressure*: See "back pressure".
+
+*primary:* Adds Processors into the result list which are set to running on the primary node only. (Regardless if the Processor is currently running or not)
+
+*running*: Adds running Ports and Processors to the result list.
+
+*stopped*: Adds stopped Ports and Processors to the result list.
+
+*timer*: Adds every Processor to the result list where the Scheduling Strategy is "Event Timer".
+
+*transmitting*: Adds Remote Process Groups to the result list which are transmitting data at the time of searching.
+
+*transmission disabled*: See "not transmitting".
+
+*transmitting enabled*: See "transmitting".
+
+*validating*: Adds Processors to the result list which are validating at the time of executing the search.
+
+=== Filters
+
+Filters provided in order to decrease the number of findings in a search based on positional information, like currently active Process Group. All filters consists of a key and a value, separated by a colon, like "_key:value_". The filters must present at the beginning of the search string and are not part of the search term. Unknown filters or known filters with unknown value are ignored. If the same filter appears multiple time, the first will be used. The order of different filters has no effect on the result.
+
+*scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "_here_". The usage of this filter looks like "_scope:here_". Any other value is considered as invalid, thus the filter will be ignored during search.
 
 Review comment:
   ```suggestion
   *scope*: This filter narrows the scope of the search based on the user's currently active Process Group. The only valid value is "here". The usage of this filter looks like "scope:here". Any other value is considered as invalid, thus the filter will be ignored during search.
   ```

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r393255610
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/test/java/org/apache/nifi/web/controller/ControllerSearchServiceTest.java
 ##########
 @@ -19,537 +19,629 @@
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
 import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.connectable.Funnel;
+import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.StandardProcessorNode;
 import org.apache.nifi.controller.flow.FlowManager;
 import org.apache.nifi.controller.label.Label;
 import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.groups.RemoteProcessGroup;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
 import org.apache.nifi.parameter.ParameterContextManager;
 import org.apache.nifi.parameter.ParameterDescriptor;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.registry.flow.StandardVersionControlInformation;
-import org.apache.nifi.registry.flow.VersionControlInformation;
-import org.apache.nifi.registry.variable.MutableVariableRegistry;
+import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricher;
+import org.apache.nifi.web.search.resultenrichment.ComponentSearchResultEnricherFactory;
+import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
-import org.mockito.AdditionalMatchers;
+import org.junit.runner.RunWith;
+import org.mockito.Mock;
 import org.mockito.Mockito;
+import org.mockito.junit.MockitoJUnitRunner;
 
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
 import java.util.Optional;
 import java.util.Set;
 
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.assertEquals;
-import static org.mockito.ArgumentMatchers.isNull;
-import static org.mockito.ArgumentMatchers.any;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-
-public class ControllerSearchServiceTest {
-    private MutableVariableRegistry variableRegistry;
-    private ControllerSearchService service;
-    private SearchResultsDTO searchResultsDTO;
+@RunWith(MockitoJUnitRunner.class)
+public class ControllerSearchServiceTest  {
+
+    public static final String PROCESS_GROUP_SECOND_LEVEL_A = "secondLevelA";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_1 = "secondLevelB1";
+    public static final String PROCESS_GROUP_SECOND_LEVEL_B_2 = "secondLevelB2";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_A = "firstLevelA";
+    public static final String PROCESS_GROUP_FIRST_LEVEL_B = "firstLevelB";
+    public static final String PROCESS_GROUP_ROOT = "root";
+
+    @Mock
+    private SearchQuery searchQuery;
+
+    @Mock
+    private NiFiUser user;
+
+    @Mock
+    private Authorizer authorizer;
+
+    @Mock
+    private ComponentSearchResultEnricherFactory resultEnricherFactory;
+
+    @Mock
+    private ComponentSearchResultEnricher resultEnricher;
+
+    @Mock
     private FlowController flowController;
+
+    @Mock
+    private FlowManager flowManager;
+
+    @Mock
     private ParameterContextManager parameterContextManager;
 
+    @Mock
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+
+    @Mock
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Connection> matcherForConnection;
+
+    @Mock
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+
+    @Mock
+    private ComponentMatcher<Port> matcherForPort;
+
+    @Mock
+    private ComponentMatcher<Funnel> matcherForFunnel;
+
+    @Mock
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+
+    @Mock
+    private ComponentMatcher<Parameter> matcherForParameter;
+
+    @Mock
+    private ComponentMatcher<Label> matcherForLabel;
+
+    private HashMap<String, ProcessGroup> processGroups;
+
+    private ControllerSearchService testSubject;
+
+    private SearchResultsDTO results;
+
     @Before
     public void setUp() {
-        variableRegistry = mock(MutableVariableRegistry.class);
-        service = new ControllerSearchService();
-        searchResultsDTO = new SearchResultsDTO();
-        flowController = mock(FlowController.class);
+        Mockito.when(resultEnricherFactory.getComponentResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getProcessGroupResultEnricher(Mockito.any(ProcessGroup.class), Mockito.any(NiFiUser.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricherFactory.getParameterResultEnricher(Mockito.any(ParameterContext.class))).thenReturn(resultEnricher);
+        Mockito.when(resultEnricher.enrich(Mockito.any(ComponentSearchResultDTO.class))).thenAnswer(invocationOnMock -> invocationOnMock.getArgument(0));
+
+        Mockito.when(matcherForProcessor.match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForProcessGroup.match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForConnection.match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForRemoteProcessGroup.match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForPort.match(Mockito.any(Port.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForFunnel.match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameterContext.match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForParameter.match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+        Mockito.when(matcherForLabel.match(Mockito.any(Label.class), Mockito.any(SearchQuery.class))).thenReturn(Optional.of(new ComponentSearchResultDTO()));
+
+        results = new SearchResultsDTO();
+        testSubject = givenTestSubject();
+    }
 
-        FlowManager mockFlowManager = mock(FlowManager.class);
-        parameterContextManager = mock(ParameterContextManager.class);
+    @Test
+    public void testSearchChecksEveryComponentType() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenAllComponentTypeIsChecked();
+        thenAllComponentResultsAreCollected();
+    }
 
-        doReturn(mockFlowManager).when(flowController).getFlowManager();
-        doReturn(parameterContextManager).when(mockFlowManager).getParameterContextManager();
-        service.setFlowController(flowController);
+    @Test
+    public void testSearchChecksChildrenGroupsToo() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
     @Test
-    public void testSearchInRootLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", rootProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("rootId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("root"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenGroupIsNotAuthorized() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenNoFilters();
+        givenProcessGroupIsNotAutorized(PROCESS_GROUP_FIRST_LEVEL_B);
+
+        // when
+        testSubject.search(searchQuery, results);
+        // The authorization is not transitive, children groups might be good candidates.
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_ROOT,
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchInThirdLevelAllAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, true, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName().equals("thirdLevelA"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWhenProcessNodeIsNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenProcessorIsNotAuthorized();
+        givenNoFilters();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenProcessorMatcherIsNotCalled();
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedNoVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() == null);
+    public void testSearchWithHereFilterShowsActualGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_A));
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A));
     }
 
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControl() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, versionControlInformation);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, null);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("firstLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName().equals("firstLevelA"));
+    public void testSearchWithHereFilterAndInRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenScopeFilterIsSet();
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
     }
 
+
     @Test
-    public void testSearchInThirdLevelParentNotAuthorizedWithVersionControlInTheGroup() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // first level PGs
-        final ProcessGroup firstLevelAProcessGroup = setupMockedProcessGroup("firstLevelA", rootProcessGroup, true, variableRegistry, null);
-        final ProcessGroup firstLevelBProcessGroup = setupMockedProcessGroup("firstLevelB", rootProcessGroup, true, variableRegistry, null);
-
-        // second level PGs
-        final ProcessGroup secondLevelAProcessGroup = setupMockedProcessGroup("secondLevelA", firstLevelAProcessGroup, true, variableRegistry, null);
-        final ProcessGroup secondLevelBProcessGroup = setupMockedProcessGroup("secondLevelB", firstLevelBProcessGroup, true, variableRegistry, null);
-        // third level PGs - not authorized
-        final VersionControlInformation versionControlInformation = setupVC();
-        final ProcessGroup thirdLevelAProcessGroup = setupMockedProcessGroup("thirdLevelA", secondLevelAProcessGroup, false, variableRegistry, versionControlInformation);
-        final ProcessGroup thirdLevelBProcessGroup = setupMockedProcessGroup("thirdLevelB", secondLevelAProcessGroup, false, variableRegistry, null);
-
-        // link PGs together
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(firstLevelAProcessGroup);
-                add(firstLevelBProcessGroup);
-            }
-        }).when(rootProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelAProcessGroup);
-            }
-        }).when(firstLevelAProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(secondLevelBProcessGroup);
-            }
-        }).when(firstLevelBProcessGroup).getProcessGroups();
-
-        Mockito.doReturn(new HashSet<ProcessGroup>() {
-            {
-                add(thirdLevelAProcessGroup);
-                add(thirdLevelBProcessGroup);
-            }
-        }).when(secondLevelAProcessGroup).getProcessGroups();
-
-        // setup processor
-        setupMockedProcessor("foobar", thirdLevelAProcessGroup, true, variableRegistry);
-
-        // perform search
-        service.search(searchResultsDTO, "foo", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getProcessorResults().size() == 1);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getId().equals("foobarId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getParentGroup().getName() == null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup() != null);
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getId().equals("thirdLevelAId"));
-        assertTrue(searchResultsDTO.getProcessorResults().get(0).getVersionedGroup().getName() == null);
+    public void testSearchWithGroupFilterShowsPointedGroupAndSubgroupsOnly() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
     }
 
     @Test
-    public void testSearchParameterContext() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", true);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testSearchGroupWithLowerCase() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Name").toLowerCase());
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupWithPartialMatch() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Na"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchGroupBasedOnIdentifier() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet((PROCESS_GROUP_FIRST_LEVEL_B + "Id"));
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList( //
+                PROCESS_GROUP_FIRST_LEVEL_B, //
+                PROCESS_GROUP_SECOND_LEVEL_B_1, //
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+    }
+
+    @Test
+    public void testSearchWithGroupWhenRoot() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet(PROCESS_GROUP_ROOT + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(
+                PROCESS_GROUP_FIRST_LEVEL_A,
+                PROCESS_GROUP_SECOND_LEVEL_A,
+                PROCESS_GROUP_FIRST_LEVEL_B,
+                PROCESS_GROUP_SECOND_LEVEL_B_1,
+                PROCESS_GROUP_SECOND_LEVEL_B_2));
+        thenContentOfTheFollowingGroupsAreSearched(processGroups.keySet());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchWithGroupWhenValueIsNonExisting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp();
+        givenGroupFilterIsSet("Unknown");
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.search(searchQuery, results);
 
-        assertEquals(1, searchResultsDTO.getParameterContextResults().size());
-        assertEquals("fooId", searchResultsDTO.getParameterContextResults().get(0).getId());
-        assertEquals("foo", searchResultsDTO.getParameterContextResults().get(0).getName());
-        // should have a match for the name, id, description
-        assertEquals(3, searchResultsDTO.getParameterContextResults().get(0).getMatches().size());
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        assertEquals(1, searchResultsDTO.getParameterResults().size());
+    @Test
+    public void testWhenBothFiltersPresentAndScopeIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_SECOND_LEVEL_B_1));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_B + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
+    }
 
-        assertEquals("fooId", searchResultsDTO.getParameterResults().get(0).getParentGroup().getId());
-        assertEquals("foo_param_0", searchResultsDTO.getParameterResults().get(0).getName());
-        // and the parameter name, parameter description, and the parameter value
-        assertEquals(3, searchResultsDTO.getParameterResults().get(0).getMatches().size());
+    @Test
+    public void testWhenBothFiltersPresentAndGroupIsMoreRestricting() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_SECOND_LEVEL_B_1 + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList(PROCESS_GROUP_SECOND_LEVEL_B_1));
     }
 
     @Test
-    public void testSearchParameterContextNotAuthorized() {
-        final ParameterContext paramContext1 = setupMockedParameterContext("foo", "description for parameter context foo", 1, "foo_param", false);
-        final ParameterContext paramContext2 = setupMockedParameterContext("bar", "description for parameter context bar", 2, "bar_param", true);
-        final Set<ParameterContext> mockedParameterContexts = new HashSet<ParameterContext>();
-        mockedParameterContexts.add(paramContext1);
-        mockedParameterContexts.add(paramContext2);
+    public void testWhenBothFiltersPresentTheyAreNotOverlapping() {
+        // given
+        givenProcessGroupsAreSetUp();
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_FIRST_LEVEL_B));
+        givenScopeFilterIsSet();
+        givenGroupFilterIsSet(PROCESS_GROUP_FIRST_LEVEL_A + "Name");
+
+        // when
+        testSubject.search(searchQuery, results);
+
+        // then
+        thenFollowingGroupsAreSearched(Arrays.asList());
+    }
 
-        Mockito.doReturn(mockedParameterContexts).when(parameterContextManager).getParameterContexts();
+    @Test
+    public void testSearchParameterContext() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(true);
 
-        service.searchParameters(searchResultsDTO, "foo");
+        // when
+        testSubject.searchParameters(searchQuery, results);
 
-        // the matching parameter context is not readable by the user, so there should not be any results
-        assertEquals(0, searchResultsDTO.getParameterContextResults().size());
-        assertEquals(0, searchResultsDTO.getParameterResults().size());
+        // then
+        thenParameterComponentTypesAreChecked();
+        thenAllParameterComponentResultsAreCollected();
     }
 
     @Test
-    public void testSearchLabels() {
-        // root level PG
-        final ProcessGroup rootProcessGroup = setupMockedProcessGroup("root", null, true, variableRegistry, null);
-
-        // setup labels
-        setupMockedLabels(rootProcessGroup);
-
-        // perform search for foo
-        service.search(searchResultsDTO, "FOO", rootProcessGroup);
-
-        assertTrue(searchResultsDTO.getLabelResults().size() == 1);
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getId().equals("foo"));
-        assertTrue(searchResultsDTO.getLabelResults().get(0).getName().equals("Value for label foo"));
-    }
-
-    /**
-     * Mocks Labels including isAuthorized() and their identifier and value
-     *
-     * @param containingProcessGroup The process group
-     */
-    private static void setupMockedLabels(final ProcessGroup containingProcessGroup) {
-        final Label label1 = mock(Label.class);
-        Mockito.doReturn(true).when(label1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("foo").when(label1).getIdentifier();
-        Mockito.doReturn("Value for label foo").when(label1).getValue();
-
-        final Label label2 = mock(Label.class);
-        Mockito.doReturn(false).when(label2).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn("bar").when(label2).getIdentifier();
-        Mockito.doReturn("Value for label bar, but FOO is in here too").when(label2).getValue();
-
-        // assign labels to the PG
-        Mockito.doReturn(new HashSet<Label>() {
-            {
-                add(label1);
-                add(label2);
-            }
-        }).when(containingProcessGroup).getLabels();
-    }
-
-    /**
-     * Sets up a mock Parameter Context including isAuthorized()
-     * @param name                     name of the parameter context
-     * @param description              description of the parameter context
-     * @param numberOfParams           number of parameters to include as part of this context
-     * @param parameterNamePrefix      a prefix for the parameter names
-     * @param authorizedToRead         whether or not the user can read the parameter context
-     * @return ParameterContext
-     */
-    private ParameterContext setupMockedParameterContext(String name, String description, int numberOfParams, String parameterNamePrefix, boolean authorizedToRead) {
-        final ParameterContext parameterContext = mock(ParameterContext.class);
-        Mockito.doReturn(name + "Id").when(parameterContext).getIdentifier();
-        Mockito.doReturn(name).when(parameterContext).getName();
-        Mockito.doReturn(description).when(parameterContext).getDescription();
-
-        Mockito.doReturn(authorizedToRead).when(parameterContext).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
-        for (int i = 0; i < numberOfParams; i++) {
-            final ParameterDescriptor descriptor = new ParameterDescriptor.Builder()
-                    .name(parameterNamePrefix + "_" + i)
-                    .description("Description for " + parameterNamePrefix + "_" + i)
-                    .sensitive(false)
-                    .build();
-
-            final Parameter param = new Parameter(descriptor, parameterNamePrefix + "_" + i + " value");
-            parameters.put(descriptor, param);
+    public void testSearchParameterContextWhenNotAuthorized() {
+        // given
+        givenSingleProcessGroupIsSetUp();
+        givenSearchQueryIsSetUp();
+        givenParameterSearchIsSetUp(false);
+
+        // when
+        testSubject.searchParameters(searchQuery, results);
+
+        // then
+        thenParameterSpecificComponentTypesAreNotChecked();
+    }
+
+    private ControllerSearchService givenTestSubject() {
+        final ControllerSearchService result = new ControllerSearchService();
+        result.setAuthorizer(authorizer);
+        result.setFlowController(flowController);
+        result.setMatcherForProcessor(matcherForProcessor);
+        result.setMatcherForProcessGroup(matcherForProcessGroup);
+        result.setMatcherForConnection(matcherForConnection);
+        result.setMatcherForRemoteProcessGroup(matcherForRemoteProcessGroup);
+        result.setMatcherForPort(matcherForPort);
+        result.setMatcherForFunnel(matcherForFunnel);
+        result.setMatcherForParameterContext(matcherForParameterContext);
+        result.setMatcherForParameter(matcherForParameter);
+        result.setMatcherForLabel(matcherForLabel);
+        result.setResultEnricherFactory(resultEnricherFactory);
+        return result;
+    }
+
+    private void givenSingleProcessGroupIsSetUp() {
+        final ProcessGroup root = givenProcessGroup(PROCESS_GROUP_ROOT, true, Collections.emptySet(), Collections.emptySet());
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+
+        final ProcessorNode processorNode = Mockito.mock(ProcessorNode.class);
+        Mockito.when(processorNode.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getProcessors()).thenReturn(Collections.singletonList(processorNode));
+
+        final Connection connection = Mockito.mock(Connection.class);
+        Mockito.when(connection.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getConnections()).thenReturn(new HashSet<>(Arrays.asList(connection)));
+
+        final RemoteProcessGroup remoteProcessGroup = Mockito.mock(RemoteProcessGroup.class);
+        Mockito.when(remoteProcessGroup.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getRemoteProcessGroups()).thenReturn(new HashSet<>(Arrays.asList(remoteProcessGroup)));
+
+        final Port port = Mockito.mock(Port.class);
+        Mockito.when(port.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getInputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+        Mockito.when(root.getOutputPorts()).thenReturn(new HashSet<>(Arrays.asList(port)));
+
+        final Funnel funnel = Mockito.mock(Funnel.class);
+        Mockito.when(funnel.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getFunnels()).thenReturn(new HashSet<>(Arrays.asList(funnel)));
+
+        final Label label = Mockito.mock(Label.class);
+        Mockito.when(label.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(true);
+        Mockito.when(root.getLabels()).thenReturn(new HashSet<>(Arrays.asList(label)));
+    }
+
+    private void givenProcessGroupsAreSetUp() {
+        final ProcessGroup secondLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_A, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB1ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_1, true, Collections.emptySet(), Collections.emptySet());
+        final ProcessGroup secondLevelB2ProcessGroup = givenProcessGroup(PROCESS_GROUP_SECOND_LEVEL_B_2, true, Collections.emptySet(), Collections.emptySet());
+
+        final ProcessGroup firstLevelAProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_A, //
+                true, Collections.emptySet(), Collections.singleton(secondLevelAProcessGroup));
+        final ProcessGroup firstLevelBProcessGroup = givenProcessGroup(PROCESS_GROUP_FIRST_LEVEL_B, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(secondLevelB1ProcessGroup, secondLevelB2ProcessGroup)));
+
+        final ProcessGroup root =  givenProcessGroup(PROCESS_GROUP_ROOT, //
+                true, Collections.emptySet(), new HashSet<>(Arrays.asList(firstLevelAProcessGroup, firstLevelBProcessGroup)));
+
+        Mockito.when(firstLevelAProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(firstLevelBProcessGroup.getParent()).thenReturn(root);
+        Mockito.when(secondLevelAProcessGroup.getParent()).thenReturn(firstLevelAProcessGroup);
+        Mockito.when(secondLevelB1ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+        Mockito.when(secondLevelB2ProcessGroup.getParent()).thenReturn(firstLevelBProcessGroup);
+
+        processGroups = new HashMap<>();
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_A, secondLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_1, secondLevelB1ProcessGroup);
+        processGroups.put(PROCESS_GROUP_SECOND_LEVEL_B_2, secondLevelB2ProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_A, firstLevelAProcessGroup);
+        processGroups.put(PROCESS_GROUP_FIRST_LEVEL_B, firstLevelBProcessGroup);
+        processGroups.put(PROCESS_GROUP_ROOT, root);
+    }
+
+    private void givenSearchQueryIsSetUp() {
+        givenSearchQueryIsSetUp(processGroups.get(PROCESS_GROUP_ROOT));
+    }
+
+    private void givenSearchQueryIsSetUp(final ProcessGroup activeProcessGroup) {
+        Mockito.when(searchQuery.getUser()).thenReturn(user);
+        Mockito.when(searchQuery.getRootGroup()).thenReturn(processGroups.get(PROCESS_GROUP_ROOT));
+        Mockito.when(searchQuery.getActiveGroup()).thenReturn(activeProcessGroup);
+    }
+
+    private ProcessGroup givenProcessGroup( //
+            final String identifier, //
+            final boolean isAuthorized, //
+            final Set<ProcessorNode> processors, //
+            final Set<ProcessGroup> children) {
+        final ProcessGroup result = Mockito.mock(ProcessGroup.class);
+        Mockito.when(result.getName()).thenReturn(identifier + "Name");
+        Mockito.when(result.getIdentifier()).thenReturn(identifier + "Id");
+        Mockito.when(result.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+
+        Mockito.when(result.getProcessGroups()).thenReturn(children);
+        Mockito.when(result.getProcessors()).thenReturn(processors);
+        Mockito.when(result.getConnections()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getRemoteProcessGroups()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getInputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getOutputPorts()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getFunnels()).thenReturn(Collections.emptySet());
+        Mockito.when(result.getLabels()).thenReturn(Collections.emptySet());
+        return result;
+    }
+
+    private void givenProcessGroupIsNotAutorized(final String processGroupName) {
+        Mockito.when(processGroups.get(processGroupName).isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenNoFilters() {
+        Mockito.when(searchQuery.hasFilter(Mockito.anyString())).thenReturn(false);
+    }
+
+    private void givenScopeFilterIsSet() {
+        Mockito.when(searchQuery.hasFilter("scope")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("scope")).thenReturn("here");
+    }
+
+    private void givenGroupFilterIsSet(final String group) {
+        Mockito.when(searchQuery.hasFilter("group")).thenReturn(true);
+        Mockito.when(searchQuery.getFilter("group")).thenReturn(group);
+    }
+
+    private void givenProcessorIsNotAuthorized() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.when(processor.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(false);
+    }
+
+    private void givenParameterSearchIsSetUp(boolean isAuthorized) {
+        final ParameterContext parameterContext = Mockito.mock(ParameterContext.class);
+        final Parameter parameter = Mockito.mock(Parameter.class);
+        final ParameterDescriptor descriptor = Mockito.mock(ParameterDescriptor.class);
+        final Map<ParameterDescriptor, Parameter> parameters = new HashMap<>();
+        parameters.put(descriptor, parameter);
+        Mockito.when(flowController.getFlowManager()).thenReturn(flowManager);
+        Mockito.when(flowManager.getParameterContextManager()).thenReturn(parameterContextManager);
+        Mockito.when(parameterContextManager.getParameterContexts()).thenReturn(new HashSet<>(Arrays.asList(parameterContext)));
+        Mockito.when(parameterContext.getParameters()).thenReturn(parameters);
+        Mockito.when(parameterContext.isAuthorized(authorizer, RequestAction.READ, user)).thenReturn(isAuthorized);
+    }
+
+    private void thenProcessorMatcherIsNotCalled() {
+        final ProcessorNode processor = processGroups.get(PROCESS_GROUP_ROOT).getProcessors().iterator().next();
+        Mockito.verify(matcherForProcessor, Mockito.never()).match(processor, searchQuery);
+    }
+
+    private void thenAllComponentTypeIsChecked() {
+//        Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForProcessor, Mockito.times(1)).match(Mockito.any(ProcessorNode.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForConnection, Mockito.times(1)).match(Mockito.any(Connection.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForRemoteProcessGroup, Mockito.times(1)).match(Mockito.any(RemoteProcessGroup.class), Mockito.any(SearchQuery.class));
+        // Port needs to be used multiple times as input and output ports are handled separately
+        Mockito.verify(matcherForPort, Mockito.times(2)).match(Mockito.any(Port.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForFunnel, Mockito.times(1)).match(Mockito.any(Funnel.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForLabel, Mockito.times(1)).match(Mockito.any(Label.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllComponentResultsAreCollected() {
+//        Assert.assertEquals(1, results.getProcessGroupResults().size());
+        Assert.assertEquals(1, results.getProcessorResults().size());
+        Assert.assertEquals(1, results.getConnectionResults().size());
+        Assert.assertEquals(1, results.getRemoteProcessGroupResults().size());
+        Assert.assertEquals(1, results.getInputPortResults().size());
+        Assert.assertEquals(1, results.getOutputPortResults().size());
+        Assert.assertEquals(1, results.getFunnelResults().size());
+        Assert.assertEquals(1, results.getLabelResults().size());
+        Assert.assertTrue(results.getParameterContextResults().isEmpty());
+        Assert.assertTrue(results.getParameterResults().isEmpty());
+    }
+
+    private void thenParameterComponentTypesAreChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.times(1)).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.times(1)).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenAllParameterComponentResultsAreCollected() {
+        Assert.assertTrue(results.getProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getProcessorResults().isEmpty());
+        Assert.assertTrue(results.getConnectionResults().isEmpty());
+        Assert.assertTrue(results.getRemoteProcessGroupResults().isEmpty());
+        Assert.assertTrue(results.getInputPortResults().isEmpty());
+        Assert.assertTrue(results.getOutputPortResults().isEmpty());
+        Assert.assertTrue(results.getFunnelResults().isEmpty());
+        Assert.assertTrue(results.getLabelResults().isEmpty());
+        Assert.assertEquals(1, results.getParameterContextResults().size());
+        Assert.assertEquals(1, results.getParameterResults().size());
+    }
+
+    private void thenParameterSpecificComponentTypesAreNotChecked() {
+        Mockito.verify(matcherForParameterContext, Mockito.never()).match(Mockito.any(ParameterContext.class), Mockito.any(SearchQuery.class));
+        Mockito.verify(matcherForParameter, Mockito.never()).match(Mockito.any(Parameter.class), Mockito.any(SearchQuery.class));
+    }
+
+    private void thenFollowingGroupsAreSearched(final Collection<String> searchedProcessGroups) {
+        for (final String processGroup : searchedProcessGroups) {
+            Mockito.verify(matcherForProcessGroup, Mockito.times(1)).match(processGroups.get(processGroup), searchQuery);
         }
 
-        Mockito.doReturn(parameters).when(parameterContext).getParameters();
-
-        return parameterContext;
-    }
-
-    /**
-     * Mocks Processor including isAuthorized() and its name & id.
-     *
-     * @param processorName          Desired processor name
-     * @param containingProcessGroup The process group
-     * @param authorizedToRead       Can the processor data be read?
-     * @param variableRegistry       The variable registry
-     */
-    private static void setupMockedProcessor(final String processorName, final ProcessGroup containingProcessGroup, boolean authorizedToRead, final MutableVariableRegistry variableRegistry) {
-        final String processorId = processorName + "Id";
-        final Processor processor1 = mock(Processor.class);
-
-        final ProcessorNode processorNode1 = mock(StandardProcessorNode.class);
-        Mockito.doReturn(authorizedToRead).when(processorNode1).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-        Mockito.doReturn(variableRegistry).when(processorNode1).getVariableRegistry();
-        Mockito.doReturn(processor1).when(processorNode1).getProcessor();
-        // set processor node's attributes
-        Mockito.doReturn(processorId).when(processorNode1).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processorNode1).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processorName).when(processorNode1).getName();
-
-        // assign processor node to its PG
-        Mockito.doReturn(new HashSet<ProcessorNode>() {
-            {
-                add(processorNode1);
-            }
-        }).when(containingProcessGroup).getProcessors();
-    }
-
-    /**
-     * Mocks ProcessGroup due to isAuthorized(). The final class StandardProcessGroup can't be used.
-     *
-     * @param processGroupName Desired process group name
-     * @param parent           The parent process group
-     * @param authorizedToRead Can the process group data be read?
-     * @param variableRegistry The variable registry
-     * @param versionControlInformation The version control information
-     * @return Mocked process group
-     */
-    private static ProcessGroup setupMockedProcessGroup(final String processGroupName, final ProcessGroup parent, boolean authorizedToRead, final VariableRegistry variableRegistry,
-                                                        final VersionControlInformation versionControlInformation) {
-        final String processGroupId = processGroupName + "Id";
-        final ProcessGroup processGroup = mock(ProcessGroup.class);
-
-        Mockito.doReturn(processGroupId).when(processGroup).getIdentifier();
-        Mockito.doReturn(Optional.ofNullable(null)).when(processGroup).getVersionedComponentId(); // not actually searching based on versioned component id
-        Mockito.doReturn(processGroupName).when(processGroup).getName();
-        Mockito.doReturn(parent).when(processGroup).getParent();
-        Mockito.doReturn(versionControlInformation).when(processGroup).getVersionControlInformation();
-        Mockito.doReturn(variableRegistry).when(processGroup).getVariableRegistry();
-        Mockito.doReturn(parent == null).when(processGroup).isRootGroup();
-        // override process group's access rights
-        Mockito.doReturn(authorizedToRead).when(processGroup).isAuthorized(AdditionalMatchers.or(any(Authorizer.class), isNull()), eq(RequestAction.READ),
-                AdditionalMatchers.or(any(NiFiUser.class), isNull()));
-
-        return processGroup;
-    }
-
-    /**
-     * Creates a version control information using dummy attributes.
-     *
-     * @return Dummy version control information
-     */
-    private static VersionControlInformation setupVC() {
-        final StandardVersionControlInformation.Builder builder = new StandardVersionControlInformation.Builder();
-        builder.registryId("regId").bucketId("bucId").flowId("flowId").version(1);
-
-        return builder.build();
-    }
-}
+        Mockito.verify(matcherForProcessGroup, Mockito.times(searchedProcessGroups.size())).match(Mockito.any(ProcessGroup.class), Mockito.any(SearchQuery.class));
 
 Review comment:
   I think this is very misleading. Feels like `matcherForProcessGroup.match` needs to be called once for every group, end then again for the same amount of times.
   
   I think this would convey the real intent better:
   `Mockito.verifyNoMoreInteractions(matcherForProcessGroup);`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391552146
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
+            if (groupName.contains(filterValue)) {
+                return true;
             }
         }
 
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getLineage(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(parameterContext.getIdentifier());
-        dto.setName(parameterContext.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchString, final Parameter parameter) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameter.getDescriptor().getName(), "Name", matches);
-        addIfAppropriate(searchString, parameter.getDescriptor().getDescription(), "Description", matches);
-        if (!parameter.getDescriptor().isSensitive()) {
-            addIfAppropriate(searchString, parameter.getValue(), "Value", matches);
-        }
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
+                .getParameterContextManager() //
+                .getParameterContexts()
+                .stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, searchQuery.getUser())) //
+                .collect(Collectors.toSet());
 
-        if (matches.isEmpty()) {
-            return null;
+        for (final ParameterContext parameterContext : parameterContexts) {
+            final SearchResultGroupDTO parentGroup = new SearchResultGroupDTO();
+            parentGroup.setId(parameterContext.getIdentifier());
+            parentGroup.setName(parameterContext.getName());
+            final MatchEnriching parameterMatchEnriching = new MatchEnriching(Optional.empty(), Optional.of(parentGroup), Optional.empty());
+
+            matcherForParameterContext.match(parameterContext, searchQuery).ifPresent(match -> results.getParameterContextResults().add(match));
 
 Review comment:
   As I see, the original implementation did not contain. It looks strange but as it has no parent or version information like other subjects of the search, it might make sense.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r401753050
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/ConnectionAttributeMatcher.java
 ##########
 @@ -0,0 +1,39 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.nifi.connectable.Connection;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class ConnectionAttributeMatcher implements AttributeMatcher<Connection> {
 
 Review comment:
   `XXXAttributeMatcher` / `XXXMatcher` are not used consistently. The shorter version would be enough for every matcher class names.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392973093
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/PortScheduledStateMatcher.java
 ##########
 @@ -0,0 +1,52 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.connectable.Port;
+import org.apache.nifi.controller.ScheduledState;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.List;
+
+public class PortScheduledStateMatcher implements AttributeMatcher<Port> {
+    private static final String SEARCH_TERM_DISABLED = "disabled";
+    private static final String SEARCH_TERM_INVALID = "invalid";
+    private static final String SEARCH_TERM_RUNNING = "running";
+    private static final String SEARCH_TERM_STOPPED = "stopped";
+
+    private static final String MATCH_PREFIX = "Run status: ";
+    private static final String MATCH_DISABLED = "Disabled";
+    private static final String MATCH_INVALID = "Invalid";
+    private static final String MATCH_RUNNING = "Running";
+    private static final String MATCH_STOPPED = "Stopped";
+
+    @Override
+    public void match(final Port component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (ScheduledState.DISABLED.equals(component.getScheduledState()) && StringUtils.containsIgnoreCase(SEARCH_TERM_DISABLED, searchTerm)) {
 
 Review comment:
   I do not know if it was deliberate but that is valid that this breaks the previous behaviour which is not the goal of this change. I ensured with an additional test that this should not silently go sideways

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r390533681
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java
 ##########
 @@ -16,564 +16,162 @@
  */
 package org.apache.nifi.web.controller;
 
-import org.apache.commons.collections4.CollectionUtils;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.nifi.authorization.Authorizer;
 import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.resource.Authorizable;
 import org.apache.nifi.authorization.user.NiFiUser;
-import org.apache.nifi.authorization.user.NiFiUserUtils;
-import org.apache.nifi.components.PropertyDescriptor;
-import org.apache.nifi.components.validation.ValidationStatus;
-import org.apache.nifi.connectable.Connectable;
 import org.apache.nifi.connectable.Connection;
 import org.apache.nifi.connectable.Funnel;
 import org.apache.nifi.connectable.Port;
 import org.apache.nifi.controller.FlowController;
 import org.apache.nifi.controller.ProcessorNode;
-import org.apache.nifi.controller.ScheduledState;
 import org.apache.nifi.controller.label.Label;
-import org.apache.nifi.controller.queue.FlowFileQueue;
-import org.apache.nifi.flowfile.FlowFilePrioritizer;
 import org.apache.nifi.groups.ProcessGroup;
 import org.apache.nifi.groups.RemoteProcessGroup;
-import org.apache.nifi.nar.NarCloseable;
 import org.apache.nifi.parameter.Parameter;
 import org.apache.nifi.parameter.ParameterContext;
-import org.apache.nifi.parameter.ParameterContextManager;
-import org.apache.nifi.processor.DataUnit;
-import org.apache.nifi.processor.Processor;
-import org.apache.nifi.processor.Relationship;
-import org.apache.nifi.registry.ComponentVariableRegistry;
-import org.apache.nifi.registry.VariableDescriptor;
-import org.apache.nifi.registry.VariableRegistry;
-import org.apache.nifi.remote.PublicPort;
-import org.apache.nifi.scheduling.ExecutionNode;
-import org.apache.nifi.scheduling.SchedulingStrategy;
-import org.apache.nifi.search.SearchContext;
-import org.apache.nifi.search.SearchResult;
-import org.apache.nifi.search.Searchable;
 import org.apache.nifi.web.api.dto.search.ComponentSearchResultDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
 import org.apache.nifi.web.api.dto.search.SearchResultsDTO;
+import org.apache.nifi.web.search.ComponentMatcher;
+import org.apache.nifi.web.search.MatchEnriching;
+import org.apache.nifi.web.search.query.SearchQuery;
 
-import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.LinkedList;
 import java.util.List;
-import java.util.Map;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
 
 /**
  * NiFi web controller's helper service that implements component search.
  */
 public class ControllerSearchService {
+    private final static String FILTER_NAME_GROUP = "group";
+    private final static String FILTER_NAME_SCOPE = "scope";
+    private final static String FILTER_SCOPE_VALUE_HERE = "here";
+
     private FlowController flowController;
     private Authorizer authorizer;
-    private VariableRegistry variableRegistry;
-
-    /**
-     * Searches term in the controller beginning from a given process group.
-     *
-     * @param results Search results
-     * @param search  The search term
-     * @param group   The init process group
-     */
-    public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-
-        if (group.isAuthorized(authorizer, RequestAction.READ, user)) {
-            final ComponentSearchResultDTO groupMatch = search(search, group);
-            if (groupMatch != null) {
-                // get the parent group, not the current one
-                groupMatch.setParentGroup(buildResultGroup(group.getParent(), user));
-                groupMatch.setVersionedGroup(buildVersionedGroup(group.getParent(), user));
-                results.getProcessGroupResults().add(groupMatch);
-            }
-        }
-
-        for (final ProcessorNode procNode : group.getProcessors()) {
-            if (procNode.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, procNode);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getProcessorResults().add(match);
-                }
-            }
-        }
-
-        for (final Connection connection : group.getConnections()) {
-            if (connection.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, connection);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getConnectionResults().add(match);
-                }
-            }
-        }
-
-        for (final RemoteProcessGroup remoteGroup : group.getRemoteProcessGroups()) {
-            if (remoteGroup.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, remoteGroup);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getRemoteProcessGroupResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getInputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getInputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Port port : group.getOutputPorts()) {
-            if (port.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, port);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getOutputPortResults().add(match);
-                }
-            }
-        }
-
-        for (final Funnel funnel : group.getFunnels()) {
-            if (funnel.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, funnel);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getFunnelResults().add(match);
-                }
-            }
-        }
-
-        for (final Label label : group.getLabels()) {
-            if (label.isAuthorized(authorizer, RequestAction.READ, user)) {
-                final ComponentSearchResultDTO match = search(search, label);
-                if (match != null) {
-                    match.setGroupId(group.getIdentifier());
-                    match.setParentGroup(buildResultGroup(group, user));
-                    match.setVersionedGroup(buildVersionedGroup(group, user));
-                    results.getLabelResults().add(match);
-                }
-            }
-        }
 
-        for (final ProcessGroup processGroup : group.getProcessGroups()) {
-            search(results, search, processGroup);
-        }
-    }
+    private ComponentMatcher<ProcessorNode> matcherForProcessor;
+    private ComponentMatcher<ProcessGroup> matcherForProcessGroup;
+    private ComponentMatcher<Connection> matcherForConnection;
+    private ComponentMatcher<RemoteProcessGroup> matcherForRemoteProcessGroup;
+    private ComponentMatcher<Port> matcherForPort;
+    private ComponentMatcher<Funnel> matcherForFunnel;
+    private ComponentMatcher<ParameterContext> matcherForParameterContext;
+    private ComponentMatcher<Parameter> matcherForParameter;
+    private ComponentMatcher<Label> matcherForLabel;
 
     /**
-     * Searches all parameter contexts and parameters
+     * Searches all parameter contexts and parameters.
+     * @param searchQuery  The search term
      * @param results Search results
-     * @param search  The search term
      */
-    public void searchParameters(final SearchResultsDTO results, final String search) {
-        final NiFiUser user = NiFiUserUtils.getNiFiUser();
-        ParameterContextManager parameterContextManager = flowController.getFlowManager().getParameterContextManager();
-
-        final Set<ParameterContext> parameterContexts = parameterContextManager.getParameterContexts();
-        for (final ParameterContext parameterContext : parameterContexts) {
-            if (parameterContext.isAuthorized(authorizer, RequestAction.READ, user)) {
-                ComponentSearchResultDTO parameterContextMatch = search(search, parameterContext);
-                if (parameterContextMatch != null) {
-                    results.getParameterContextResults().add(parameterContextMatch);
-                }
-
-                // search each parameter within the context as well
-                for (Parameter parameter : parameterContext.getParameters().values()) {
-                    ComponentSearchResultDTO parameterMatch = search(search, parameter);
-                    if (parameterMatch != null) {
-                        final SearchResultGroupDTO paramContextGroup = new SearchResultGroupDTO();
-                        paramContextGroup.setId(parameterContext.getIdentifier());
-                        paramContextGroup.setName(parameterContext.getName());
-                        parameterMatch.setParentGroup(paramContextGroup);
-
-                        results.getParameterResults().add(parameterMatch);
-                    }
-                }
-            }
-        }
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Port port) {
-        final List<String> matches = new ArrayList<>();
-
-        addIfAppropriate(searchStr, port.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, port.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, port.getName(), "Name", matches);
-        addIfAppropriate(searchStr, port.getComments(), "Comments", matches);
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(port.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
+    public void search(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        if (searchQuery.hasFilter(FILTER_NAME_SCOPE) && FILTER_SCOPE_VALUE_HERE.equals(searchQuery.getFilter(FILTER_NAME_SCOPE))) {
+            searchInProcessGroup(results, searchQuery, searchQuery.getActiveGroup());
         } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && !port.isValid()) {
-                matches.add("Run status: Invalid");
-            } else if (ScheduledState.RUNNING.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(port.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        if (port instanceof PublicPort) {
-            final PublicPort publicPort = (PublicPort) port;
-
-            // user access controls
-            for (final String userAccessControl : publicPort.getUserAccessControl()) {
-                addIfAppropriate(searchStr, userAccessControl, "User access control", matches);
-            }
-
-            // group access controls
-            for (final String groupAccessControl : publicPort.getGroupAccessControl()) {
-                addIfAppropriate(searchStr, groupAccessControl, "Group access control", matches);
-            }
+            searchInProcessGroup(results, searchQuery, searchQuery.getRootGroup());
         }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(port.getIdentifier());
-        dto.setName(port.getName());
-        dto.setMatches(matches);
-        return dto;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessorNode procNode) {
-        final List<String> matches = new ArrayList<>();
-        final Processor processor = procNode.getProcessor();
-
-        addIfAppropriate(searchStr, procNode.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, procNode.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, procNode.getName(), "Name", matches);
-        addIfAppropriate(searchStr, procNode.getComments(), "Comments", matches);
-
-        // consider scheduling strategy
-        if (SchedulingStrategy.EVENT_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("event", searchStr)) {
-            matches.add("Scheduling strategy: Event driven");
-        } else if (SchedulingStrategy.TIMER_DRIVEN.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("timer", searchStr)) {
-            matches.add("Scheduling strategy: Timer driven");
-        } else if (SchedulingStrategy.PRIMARY_NODE_ONLY.equals(procNode.getSchedulingStrategy()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            // PRIMARY_NODE_ONLY has been deprecated as a SchedulingStrategy and replaced by PRIMARY as an ExecutionNode.
-            matches.add("Scheduling strategy: On primary node");
-        }
-
-        // consider execution node
-        if (ExecutionNode.PRIMARY.equals(procNode.getExecutionNode()) && StringUtils.containsIgnoreCase("primary", searchStr)) {
-            matches.add("Execution node: primary");
-        }
-
-        // consider scheduled state
-        if (ScheduledState.DISABLED.equals(procNode.getScheduledState())) {
-            if (StringUtils.containsIgnoreCase("disabled", searchStr)) {
-                matches.add("Run status: Disabled");
-            }
-        } else {
-            if (StringUtils.containsIgnoreCase("invalid", searchStr) && procNode.getValidationStatus() == ValidationStatus.INVALID) {
-                matches.add("Run status: Invalid");
-            } else if (StringUtils.containsIgnoreCase("validating", searchStr) && procNode.getValidationStatus() == ValidationStatus.VALIDATING) {
-                matches.add("Run status: Validating");
-            } else if (ScheduledState.RUNNING.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("running", searchStr)) {
-                matches.add("Run status: Running");
-            } else if (ScheduledState.STOPPED.equals(procNode.getScheduledState()) && StringUtils.containsIgnoreCase("stopped", searchStr)) {
-                matches.add("Run status: Stopped");
-            }
-        }
-
-        for (final Relationship relationship : procNode.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // Add both the actual class name and the component type. This allows us to search for 'Ghost'
-        // to search for components that could not be instantiated.
-        addIfAppropriate(searchStr, processor.getClass().getSimpleName(), "Type", matches);
-        addIfAppropriate(searchStr, procNode.getComponentType(), "Type", matches);
-
-        for (final Map.Entry<PropertyDescriptor, String> entry : procNode.getRawPropertyValues().entrySet()) {
-            final PropertyDescriptor descriptor = entry.getKey();
-
-            addIfAppropriate(searchStr, descriptor.getName(), "Property name", matches);
-            addIfAppropriate(searchStr, descriptor.getDescription(), "Property description", matches);
-
-            // never include sensitive properties values in search results
-            if (descriptor.isSensitive()) {
-                continue;
-            }
-
-            String value = entry.getValue();
-
-            // if unset consider default value
-            if (value == null) {
-                value = descriptor.getDefaultValue();
-            }
-
-            // evaluate if the value matches the search criteria
-            if (StringUtils.containsIgnoreCase(value, searchStr)) {
-                matches.add("Property value: " + descriptor.getName() + " - " + value);
-            }
-        }
-
-        // consider searching the processor directly
-        if (processor instanceof Searchable) {
-            final Searchable searchable = (Searchable) processor;
-
-            final SearchContext context = new StandardSearchContext(searchStr, procNode, flowController.getControllerServiceProvider(), variableRegistry);
-
-            // search the processor using the appropriate thread context classloader
-            try (final NarCloseable x = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), processor.getClass(), processor.getIdentifier())) {
-                final Collection<SearchResult> searchResults = searchable.search(context);
-                if (CollectionUtils.isNotEmpty(searchResults)) {
-                    for (final SearchResult searchResult : searchResults) {
-                        matches.add(searchResult.getLabel() + ": " + searchResult.getMatch());
-                    }
-                }
-            } catch (final Throwable t) {
-                // log this as error
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(procNode.getIdentifier());
-        result.setMatches(matches);
-        result.setName(procNode.getName());
-        return result;
+    private void searchInProcessGroup(final SearchResultsDTO results, final SearchQuery searchQuery, final ProcessGroup scope) {
+        final NiFiUser user = searchQuery.getUser();
+        final SearchResultGroupDTO parentGroup = buildResultGroup(scope, user);
+        final SearchResultGroupDTO versionedGroup = buildVersionedGroup(scope, user);
+        final MatchEnriching matchEnriching = new MatchEnriching(scope.getIdentifier(), parentGroup, versionedGroup);
+        final MatchEnriching partialMatchEnriching = new MatchEnriching(Optional.empty(), Optional.ofNullable(parentGroup), Optional.ofNullable(versionedGroup));
+
+        if (appliesToGroupFilter(searchQuery, scope)) {
+            searchComponentType(Collections.singletonList(scope), user, searchQuery, matcherForProcessGroup, partialMatchEnriching, results.getProcessGroupResults());
+            searchComponentType(scope.getProcessors(), user, searchQuery, matcherForProcessor, matchEnriching, results.getProcessorResults());
+            searchComponentType(scope.getConnections(), user, searchQuery, matcherForConnection, matchEnriching, results.getConnectionResults());
+            searchComponentType(scope.getRemoteProcessGroups(), user, searchQuery, matcherForRemoteProcessGroup, matchEnriching, results.getRemoteProcessGroupResults());
+            searchComponentType(scope.getInputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getInputPortResults());
+            searchComponentType(scope.getOutputPorts(), user, searchQuery, matcherForPort, matchEnriching, results.getOutputPortResults());
+            searchComponentType(scope.getFunnels(), user, searchQuery, matcherForFunnel, matchEnriching, results.getFunnelResults());
+            searchComponentType(scope.getLabels(), user, searchQuery, matcherForLabel, matchEnriching, results.getLabelResults());
+        }
+
+        scope.getProcessGroups().forEach(processGroup -> searchInProcessGroup(results, searchQuery, processGroup));
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final ProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        final ProcessGroup parent = group.getParent();
-        if (parent == null) {
-            return null;
-        }
-
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-
-        final ComponentVariableRegistry varRegistry = group.getVariableRegistry();
-        if (varRegistry != null) {
-            final Map<VariableDescriptor, String> variableMap = varRegistry.getVariableMap();
-            for (final Map.Entry<VariableDescriptor, String> entry : variableMap.entrySet()) {
-                addIfAppropriate(searchStr, entry.getKey().getName(), "Variable Name", matches);
-                addIfAppropriate(searchStr, entry.getValue(), "Variable Value", matches);
-            }
-        }
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setGroupId(parent.getIdentifier());
-        result.setMatches(matches);
-        return result;
+    private boolean appliesToGroupFilter(final SearchQuery searchQuery, final ProcessGroup scope) {
+        return searchQuery.hasFilter(FILTER_NAME_GROUP)
+                ? eligibleForGroupFilter(scope, searchQuery.getFilter(FILTER_NAME_GROUP))
+                : true;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Connection connection) {
-        final List<String> matches = new ArrayList<>();
-
-        // search id and name
-        addIfAppropriate(searchStr, connection.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, connection.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, connection.getName(), "Name", matches);
-
-        // search relationships
-        for (final Relationship relationship : connection.getRelationships()) {
-            addIfAppropriate(searchStr, relationship.getName(), "Relationship", matches);
-        }
-
-        // search prioritizers
-        final FlowFileQueue queue = connection.getFlowFileQueue();
-        for (final FlowFilePrioritizer comparator : queue.getPriorities()) {
-            addIfAppropriate(searchStr, comparator.getClass().getName(), "Prioritizer", matches);
-        }
-
-        // search expiration
-        if (StringUtils.containsIgnoreCase("expires", searchStr) || StringUtils.containsIgnoreCase("expiration", searchStr)) {
-            final int expirationMillis = connection.getFlowFileQueue().getFlowFileExpiration(TimeUnit.MILLISECONDS);
-            if (expirationMillis > 0) {
-                matches.add("FlowFile expiration: " + connection.getFlowFileQueue().getFlowFileExpiration());
-            }
-        }
-
-        // search back pressure
-        if (StringUtils.containsIgnoreCase("back pressure", searchStr) || StringUtils.containsIgnoreCase("pressure", searchStr)) {
-            final String backPressureDataSize = connection.getFlowFileQueue().getBackPressureDataSizeThreshold();
-            final Double backPressureBytes = DataUnit.parseDataSize(backPressureDataSize, DataUnit.B);
-            if (backPressureBytes > 0) {
-                matches.add("Back pressure data size: " + backPressureDataSize);
-            }
-
-            final long backPressureCount = connection.getFlowFileQueue().getBackPressureObjectThreshold();
-            if (backPressureCount > 0) {
-                matches.add("Back pressure count: " + backPressureCount);
-            }
-        }
-
-        // search the source
-        final Connectable source = connection.getSource();
-        addIfAppropriate(searchStr, source.getIdentifier(), "Source id", matches);
-        addIfAppropriate(searchStr, source.getName(), "Source name", matches);
-        addIfAppropriate(searchStr, source.getComments(), "Source comments", matches);
-
-        // search the destination
-        final Connectable destination = connection.getDestination();
-        addIfAppropriate(searchStr, destination.getIdentifier(), "Destination id", matches);
-        addIfAppropriate(searchStr, destination.getName(), "Destination name", matches);
-        addIfAppropriate(searchStr, destination.getComments(), "Destination comments", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
+    private boolean eligibleForGroupFilter(final ProcessGroup scope, final String filterValue) {
+        final List<String> lineage = getLineage(scope);
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(connection.getIdentifier());
-
-        // determine the name of the search match
-        if (StringUtils.isNotBlank(connection.getName())) {
-            result.setName(connection.getName());
-        } else if (!connection.getRelationships().isEmpty()) {
-            final List<String> relationships = new ArrayList<>(connection.getRelationships().size());
-            for (final Relationship relationship : connection.getRelationships()) {
-                if (StringUtils.isNotBlank(relationship.getName())) {
-                    relationships.add(relationship.getName());
-                }
-            }
-            if (!relationships.isEmpty()) {
-                result.setName(StringUtils.join(relationships, ", "));
+        for (final String groupName : lineage) {
+            if (groupName.contains(filterValue)) {
+                return true;
             }
         }
 
-        // ensure a name is added
-        if (result.getName() == null) {
-            result.setName("From source " + connection.getSource().getName());
-        }
-
-        result.setMatches(matches);
-        return result;
+        return false;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final RemoteProcessGroup group) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, group.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, group.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-        addIfAppropriate(searchStr, group.getName(), "Name", matches);
-        addIfAppropriate(searchStr, group.getComments(), "Comments", matches);
-        addIfAppropriate(searchStr, group.getTargetUris(), "URLs", matches);
-
-        // consider the transmission status
-        if ((StringUtils.containsIgnoreCase("transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission enabled", searchStr)) && group.isTransmitting()) {
-            matches.add("Transmission: On");
-        } else if ((StringUtils.containsIgnoreCase("not transmitting", searchStr) || StringUtils.containsIgnoreCase("transmission disabled", searchStr)) && !group.isTransmitting()) {
-            matches.add("Transmission: Off");
-        }
+    private List<String> getLineage(final ProcessGroup group) {
+        final LinkedList<String> result = new LinkedList<>();
+        result.add(group.getName());
+        ProcessGroup actual = group;
 
-        if (matches.isEmpty()) {
-            return null;
+        while (actual.getParent() != null) {
+            actual = actual.getParent();
+            result.addLast(actual.getName());
         }
 
-        final ComponentSearchResultDTO result = new ComponentSearchResultDTO();
-        result.setId(group.getIdentifier());
-        result.setName(group.getName());
-        result.setMatches(matches);
         return result;
     }
 
-    private ComponentSearchResultDTO search(final String searchStr, final Funnel funnel) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, funnel.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, funnel.getVersionedComponentId().orElse(null), "Version Control ID", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(funnel.getIdentifier());
-        dto.setName(funnel.getName());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchStr, final Label label) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchStr, label.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchStr, label.getValue(), "Value", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(label.getIdentifier());
-        dto.setName(label.getValue());
-        dto.setMatches(matches);
-        return dto;
-    }
-
-    private ComponentSearchResultDTO search(final String searchString, final ParameterContext parameterContext) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameterContext.getIdentifier(), "Id", matches);
-        addIfAppropriate(searchString, parameterContext.getName(), "Name", matches);
-        addIfAppropriate(searchString, parameterContext.getDescription(), "Description", matches);
-
-        if (matches.isEmpty()) {
-            return null;
-        }
-
-        final ComponentSearchResultDTO dto = new ComponentSearchResultDTO();
-        dto.setId(parameterContext.getIdentifier());
-        dto.setName(parameterContext.getName());
-        dto.setMatches(matches);
-        return dto;
+    private <T extends Authorizable> void searchComponentType( //
+            final Collection<T> components, //
+            final NiFiUser user, //
+            final SearchQuery searchQuery, //
+            final ComponentMatcher<T> matcher, //
+            final MatchEnriching matchEnriching, //
+            final List<ComponentSearchResultDTO> resultAccumulator) {
+        components.stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, user)) //
+                .map(component -> matcher.match(component, searchQuery)) //
+                .filter(result -> result.isPresent()) //
+                .map(result -> matchEnriching.apply(result.get())) //
+                .forEach(result -> resultAccumulator.add(result));
     }
 
-    private ComponentSearchResultDTO search(final String searchString, final Parameter parameter) {
-        final List<String> matches = new ArrayList<>();
-        addIfAppropriate(searchString, parameter.getDescriptor().getName(), "Name", matches);
-        addIfAppropriate(searchString, parameter.getDescriptor().getDescription(), "Description", matches);
-        if (!parameter.getDescriptor().isSensitive()) {
-            addIfAppropriate(searchString, parameter.getValue(), "Value", matches);
-        }
+    /**
+     * Searches all parameter contexts and parameters
+     * @param searchQuery Details of the search
+     * @param results Search results
+     */
+    public void searchParameters(final SearchQuery searchQuery, final SearchResultsDTO results) {
+        final Set<ParameterContext> parameterContexts = flowController.getFlowManager() //
+                .getParameterContextManager() //
+                .getParameterContexts()
+                .stream() //
+                .filter(component -> component.isAuthorized(authorizer, RequestAction.READ, searchQuery.getUser())) //
+                .collect(Collectors.toSet());
 
-        if (matches.isEmpty()) {
-            return null;
+        for (final ParameterContext parameterContext : parameterContexts) {
+            final SearchResultGroupDTO parentGroup = new SearchResultGroupDTO();
+            parentGroup.setId(parameterContext.getIdentifier());
+            parentGroup.setName(parameterContext.getName());
+            final MatchEnriching parameterMatchEnriching = new MatchEnriching(Optional.empty(), Optional.of(parentGroup), Optional.empty());
+
+            matcherForParameterContext.match(parameterContext, searchQuery).ifPresent(match -> results.getParameterContextResults().add(match));
+            parameterContext.getParameters().values().stream()
+                    .map(component -> matcherForParameter.match(component, searchQuery))
+                    .filter(result -> result.isPresent()) //
+                    .map(result -> parameterMatchEnriching.apply(result.get()))
 
 Review comment:
   The same `parentGroup` is used for all match results. It's probably fine, but I feel uncomfortable about it because `SearchResultGroupDTO` is _not_ immutable.
   I think we should _not_ try to save the creation of those objects unless we can responsibly change it to be immutable.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391141901
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/PropertyMatcher.java
 ##########
 @@ -0,0 +1,68 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.web.search.query.SearchQuery;
+
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.apache.nifi.web.search.attributematchers.AttributeMatcher.addIfMatching;
+
+public class PropertyMatcher implements AttributeMatcher<ProcessorNode> {
+    private static final String LABEL_NAME = "Property name";
+    private static final String LABEL_VALUE = "Property value";
+    private static final String LABEL_DESCRIPTION = "Property description";
+
+    private final static String FILTER_NAME_PROPERTIES = "properties";
+    private final static Set<String> FILTER_VALUES_PROPERTIES_EXCLUSION = new HashSet<>(Arrays.asList("no", "none", "false", "exclude", "0"));
+
+    @Override
+    public void match(final ProcessorNode component, final SearchQuery query, final List<String> matches) {
+        final String searchTerm = query.getTerm();
+
+        if (!propertiesAreFilteredOut(query)) {
+            for (final Map.Entry<PropertyDescriptor, String> entry : component.getRawPropertyValues().entrySet()) {
+                final PropertyDescriptor descriptor = entry.getKey();
+                addIfMatching(searchTerm, descriptor.getName(), LABEL_NAME, matches);
+                addIfMatching(searchTerm, descriptor.getDescription(), LABEL_DESCRIPTION, matches);
+
+                // never include sensitive properties values in search results
+                if (!descriptor.isSensitive()) {
+                    final String value = (entry.getValue() == null)
+                        ? descriptor.getDefaultValue()
+                        : entry.getValue();
+
+                    // evaluate if the value matches the search criteria
+                    if (StringUtils.containsIgnoreCase(value, searchTerm)) {
+                        matches.add(new StringBuilder(LABEL_VALUE).append(SEPARATOR).append(descriptor.getName()).append(" - ").append(value).toString());
 
 Review comment:
   Again, unnecessary `StringBuilder`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r402149812
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/resultenrichment/AbstractComponentSearchResultEnricher.java
 ##########
 @@ -0,0 +1,87 @@
+/*
+ * 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.web.search.resultenrichment;
+
+import org.apache.nifi.authorization.Authorizer;
+import org.apache.nifi.authorization.RequestAction;
+import org.apache.nifi.authorization.user.NiFiUser;
+import org.apache.nifi.groups.ProcessGroup;
+import org.apache.nifi.web.api.dto.search.SearchResultGroupDTO;
+
+abstract class AbstractComponentSearchResultEnricher implements ComponentSearchResultEnricher {
+    protected final ProcessGroup processGroup;
+    protected final NiFiUser user;
+    protected final Authorizer authorizer;
+
+    AbstractComponentSearchResultEnricher(final ProcessGroup processGroup, final NiFiUser user, final Authorizer authorizer) {
+        this.processGroup = processGroup;
+        this.user = user;
+        this.authorizer = authorizer;
+    }
+
+    /**
+     * Builds the nearest versioned parent result group for a given user.
+     *
+     * @param group The containing group
+     * @param user The current NiFi user
+     * @return Versioned parent group
+     */
+    protected SearchResultGroupDTO buildVersionedGroup(final ProcessGroup group, final NiFiUser user) {
+        if (group == null) {
+            return null;
+        }
+
+        ProcessGroup tmpParent = group.getParent();
+        ProcessGroup tmpGroup = group;
+
+        // search for a versioned group by traversing the group tree up to the root
+        while (!tmpGroup.isRootGroup()) {
+            if (tmpGroup.getVersionControlInformation() != null) {
+                return buildResultGroup(tmpGroup, user);
+            }
+
+            tmpGroup = tmpParent;
+            tmpParent = tmpGroup.getParent();
+        }
 
 Review comment:
   Copied from previous implementation but makes sense, thanks

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
simonbence commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r396091198
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-canvas-utils.js
 ##########
 @@ -2118,6 +2118,13 @@
             return nfCanvas.getParentGroupId();
         },
 
+        /**
+         * Get the group id.
+         */
+        getGroupId: function () {
 
 Review comment:
   Thanks, I removed the duplication

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r392218227
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerFacade.java
 ##########
 @@ -1614,15 +1615,20 @@ private void setComponentDetails(final ProvenanceEventDTO dto) {
     /**
      * Searches this controller for the specified term.
      *
-     * @param search search
+     * @param searchLiteral search
 
 Review comment:
   I see "search" is the old comment, but could you please provide more context here: "search string specified by the user" or similar (if I understand correctly it is the user input)

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [nifi] tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #4123: NIFI-7188: Adding filter capabilities into search & prerequisite refactors
URL: https://github.com/apache/nifi/pull/4123#discussion_r391149208
 
 

 ##########
 File path: nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/search/attributematchers/SearchableMatcher.java
 ##########
 @@ -0,0 +1,74 @@
+/*
+ * 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.web.search.attributematchers;
+
+import org.apache.commons.collections4.CollectionUtils;
+import org.apache.nifi.controller.FlowController;
+import org.apache.nifi.controller.ProcessorNode;
+import org.apache.nifi.nar.NarCloseable;
+import org.apache.nifi.processor.Processor;
+import org.apache.nifi.registry.VariableRegistry;
+import org.apache.nifi.search.SearchContext;
+import org.apache.nifi.search.SearchResult;
+import org.apache.nifi.search.Searchable;
+import org.apache.nifi.web.controller.StandardSearchContext;
+import org.apache.nifi.web.search.query.SearchQuery;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Collection;
+import java.util.List;
+
+public class SearchableMatcher implements AttributeMatcher<ProcessorNode> {
+    private static final Logger LOGGER = LoggerFactory.getLogger(SearchableMatcher.class);
+
+    private FlowController flowController;
+    private VariableRegistry variableRegistry;
+
+    @Override
+    public void match(final ProcessorNode component, final SearchQuery query, final List<String> matches) {
+        final Processor processor = component.getProcessor();
+
+        if (processor instanceof Searchable) {
+            final Searchable searchable = (Searchable) processor;
+            final String searchTerm = query.getTerm();
+            final SearchContext context = new StandardSearchContext(searchTerm, component, flowController.getControllerServiceProvider(), variableRegistry);
+
+            // search the processor using the appropriate thread context classloader
+            try (final NarCloseable narCloseable = NarCloseable.withComponentNarLoader(flowController.getExtensionManager(), component.getClass(), component.getIdentifier())) {
+                final Collection<SearchResult> searchResults = searchable.search(context);
+
+                if (CollectionUtils.isNotEmpty(searchResults)) {
+                    for (final SearchResult searchResult : searchResults) {
+                        matches.add(new StringBuilder(searchResult.getLabel()).append(AttributeMatcher.SEPARATOR).append(searchResult.getMatch()).toString());
 
 Review comment:
   Unnecessary `StringBuilder`.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services