You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by "EndzeitBegins (via GitHub)" <gi...@apache.org> on 2023/11/26 13:35:15 UTC

[PR] NIFI-12386 Adds processor FilterAttributes (Java 1.8 / NiFi 1.x) [nifi]

EndzeitBegins opened a new pull request, #8064:
URL: https://github.com/apache/nifi/pull/8064

   <!-- 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. -->
   
   # Summary
   
   Adds a new processor `FilterAttributes` to the `nifi-standard-processors` bundle.
   See [NIFI-12386](https://issues.apache.org/jira/browse/NIFI-12386).
   
   This is backport of #8049 compatible with Java 1.8 / NiFi 1.x.
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [x] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [x] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [x] Pull Request based on current revision of the `main` branch
   - [x] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [x] Build completed using `mvn clean install -P contrib-check`
     - [x] JDK ~~21~~ 8
   
   ### Licensing
   
   - [x] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [x] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [x] Documentation formatting appears as expected in rendered 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "mosermw (via GitHub)" <gi...@apache.org>.
mosermw commented on PR #8064:
URL: https://github.com/apache/nifi/pull/8064#issuecomment-1834609921

   I've finished testing on the support/nifi-1.x branch and this processor works in the supported scenarios.  So +1 I will merge.  We should have plenty of time to resolve NIFI-12446 before the next release(s).
   
   As you can probably tell, my reviews tend to be lenient when it comes to code style, as long as readability, functionality and performance goals 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411197372


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")

Review Comment:
   Title Case should be used for property and display names:
   ```suggestion
               .name("Attribute Matching Strategy")
               .displayName("Attribute Matching Strategy")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private volatile Predicate<String> cachedMatchingPredicate;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        cachedMatchingPredicate = null;
+
+        if (matchingStrategy == MatchingStrategy.ENUMERATION
+                && !context.getProperty(ATTRIBUTE_SET).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnEnumeration(context, null);
+        }
+        if (matchingStrategy == MatchingStrategy.REGEX
+                && !context.getProperty(ATTRIBUTE_REGEX).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnRegex(context, null);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final Predicate<String> matchingPredicate = determineMatchingPredicate(context, flowFile);
+
+        final FilterMode filterMode = getFilterMode(context);
+        final Predicate<String> isMatched;
+        if (filterMode == FilterMode.RETAIN) {
+            isMatched = matchingPredicate;
+        } else {
+            isMatched = matchingPredicate.negate();
+        }
+
+        final Set<String> attributesToRemove = new HashSet<>(flowFile.getAttributes().keySet());
+        attributesToRemove.removeIf(isMatched);
+
+        final FlowFile updatedFlowFile = session.removeAllAttributes(flowFile, attributesToRemove);
+        session.transfer(updatedFlowFile, REL_SUCCESS);
+    }
+
+    private Predicate<String> determineMatchingPredicate(ProcessContext context, FlowFile flowFile) {
+        if (cachedMatchingPredicate != null) {
+            return cachedMatchingPredicate;
+        }
+
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        switch (matchingStrategy) {
+            case ENUMERATION:
+                return determineMatchingPredicateBasedOnEnumeration(context, flowFile);
+            case REGEX:
+                return determineMatchingPredicateBasedOnRegex(context, flowFile);
+            default:
+                throw new IllegalArgumentException(
+                        "Cannot determine matching predicate for unsupported strategy " + matchingStrategy + " !"
+                );
+        }
+    }
+
+    private static Predicate<String> determineMatchingPredicateBasedOnEnumeration(ProcessContext context, FlowFile flowFile) {
+        final String attributeSetDeclaration = getAttributeSet(context, flowFile);
+        final String delimiter = getDelimiter();
+
+        Set<String> attributeSet = Arrays.stream(attributeSetDeclaration.split(Pattern.quote(delimiter)))
+                .map(String::trim)
+                .filter(attributeName -> !attributeName.trim().isEmpty())

Review Comment:
   Is it possible to use the `not()` Predicate matcher instead of the `!` in this case?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")

Review Comment:
   Recommend naming this `Filtered Attributes`:
   ```suggestion
               .name("Filtered Attributes")
               .displayName("Filtered Attributes")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private volatile Predicate<String> cachedMatchingPredicate;

Review Comment:
   This should be moved up above the method declarations.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private volatile Predicate<String> cachedMatchingPredicate;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        cachedMatchingPredicate = null;
+
+        if (matchingStrategy == MatchingStrategy.ENUMERATION
+                && !context.getProperty(ATTRIBUTE_SET).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnEnumeration(context, null);
+        }
+        if (matchingStrategy == MatchingStrategy.REGEX
+                && !context.getProperty(ATTRIBUTE_REGEX).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnRegex(context, null);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final Predicate<String> matchingPredicate = determineMatchingPredicate(context, flowFile);
+
+        final FilterMode filterMode = getFilterMode(context);
+        final Predicate<String> isMatched;
+        if (filterMode == FilterMode.RETAIN) {
+            isMatched = matchingPredicate;
+        } else {
+            isMatched = matchingPredicate.negate();
+        }
+
+        final Set<String> attributesToRemove = new HashSet<>(flowFile.getAttributes().keySet());
+        attributesToRemove.removeIf(isMatched);
+
+        final FlowFile updatedFlowFile = session.removeAllAttributes(flowFile, attributesToRemove);
+        session.transfer(updatedFlowFile, REL_SUCCESS);
+    }
+
+    private Predicate<String> determineMatchingPredicate(ProcessContext context, FlowFile flowFile) {
+        if (cachedMatchingPredicate != null) {
+            return cachedMatchingPredicate;
+        }
+
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        switch (matchingStrategy) {
+            case ENUMERATION:
+                return determineMatchingPredicateBasedOnEnumeration(context, flowFile);
+            case REGEX:
+                return determineMatchingPredicateBasedOnRegex(context, flowFile);
+            default:
+                throw new IllegalArgumentException(
+                        "Cannot determine matching predicate for unsupported strategy " + matchingStrategy + " !"

Review Comment:
   Exclamation marks should not be used in messages.
   ```suggestion
                           "Cannot determine matching predicate for unsupported strategy [" + matchingStrategy + "]"
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   Instead of using the String `RETAIN`, recommend using the enum value with `toString()` to avoid any potential misalignment between the enum value and the `AllowableValue`.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")

Review Comment:
   ```suggestion
               .name("Filtered Attributes Pattern")
               .displayName("Filtered Attributes Pattern")
   ```



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);
+    }
+
+    @Override
+    public Set<Relationship> getRelationships() {
+        return relationships;
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    private volatile Predicate<String> cachedMatchingPredicate;
+
+    @OnScheduled
+    public void onScheduled(final ProcessContext context) {
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        cachedMatchingPredicate = null;
+
+        if (matchingStrategy == MatchingStrategy.ENUMERATION
+                && !context.getProperty(ATTRIBUTE_SET).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnEnumeration(context, null);
+        }
+        if (matchingStrategy == MatchingStrategy.REGEX
+                && !context.getProperty(ATTRIBUTE_REGEX).isExpressionLanguagePresent()) {
+            cachedMatchingPredicate = determineMatchingPredicateBasedOnRegex(context, null);
+        }
+    }
+
+    @Override
+    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
+        final FlowFile flowFile = session.get();
+        if (flowFile == null) {
+            return;
+        }
+
+        final Predicate<String> matchingPredicate = determineMatchingPredicate(context, flowFile);
+
+        final FilterMode filterMode = getFilterMode(context);
+        final Predicate<String> isMatched;
+        if (filterMode == FilterMode.RETAIN) {
+            isMatched = matchingPredicate;
+        } else {
+            isMatched = matchingPredicate.negate();
+        }
+
+        final Set<String> attributesToRemove = new HashSet<>(flowFile.getAttributes().keySet());
+        attributesToRemove.removeIf(isMatched);
+
+        final FlowFile updatedFlowFile = session.removeAllAttributes(flowFile, attributesToRemove);
+        session.transfer(updatedFlowFile, REL_SUCCESS);
+    }
+
+    private Predicate<String> determineMatchingPredicate(ProcessContext context, FlowFile flowFile) {
+        if (cachedMatchingPredicate != null) {
+            return cachedMatchingPredicate;
+        }
+
+        final MatchingStrategy matchingStrategy = getMatchingStrategy(context);
+
+        switch (matchingStrategy) {
+            case ENUMERATION:
+                return determineMatchingPredicateBasedOnEnumeration(context, flowFile);
+            case REGEX:
+                return determineMatchingPredicateBasedOnRegex(context, flowFile);
+            default:
+                throw new IllegalArgumentException(
+                        "Cannot determine matching predicate for unsupported strategy " + matchingStrategy + " !"
+                );
+        }
+    }
+
+    private static Predicate<String> determineMatchingPredicateBasedOnEnumeration(ProcessContext context, FlowFile flowFile) {
+        final String attributeSetDeclaration = getAttributeSet(context, flowFile);
+        final String delimiter = getDelimiter();
+
+        Set<String> attributeSet = Arrays.stream(attributeSetDeclaration.split(Pattern.quote(delimiter)))
+                .map(String::trim)
+                .filter(attributeName -> !attributeName.trim().isEmpty())
+                .collect(Collectors.toSet());
+
+        return attributeSet::contains;
+    }
+
+    private static Predicate<String> determineMatchingPredicateBasedOnRegex(ProcessContext context, FlowFile flowFile) {
+        Pattern attributeRegex = getAttributeRegex(context, flowFile);
+
+        return attributeName -> attributeRegex.matcher(attributeName).matches();
+    }
+
+    /* properties */
+
+    private static FilterMode getFilterMode(ProcessContext context) {
+        final String rawFilterMode = context
+                .getProperty(FILTER_MODE)
+                .getValue();
+
+        if (FILTER_MODE_VALUE_REMOVE.getValue().equals(rawFilterMode)) {
+            return FilterMode.REMOVE;
+        }
+        return FilterMode.RETAIN;
+    }
+
+    private static MatchingStrategy getMatchingStrategy(ProcessContext context) {
+        final String rawMatchingStrategy = context
+                .getProperty(MATCHING_STRATEGY)
+                .getValue();
+
+        if (MATCHING_STRATEGY_VALUE_REGEX.getValue().equals(rawMatchingStrategy)) {
+            return MatchingStrategy.REGEX;
+        }
+        return MatchingStrategy.ENUMERATION;
+    }
+
+    private static String getAttributeSet(ProcessContext context, FlowFile flowFile) {
+        return context.getProperty(ATTRIBUTE_SET).evaluateAttributeExpressions(flowFile).getValue();
+    }
+
+    private static String getDelimiter() {
+        return DELIMITER_VALUE;
+    }
+
+    private static Pattern getAttributeRegex(ProcessContext context, FlowFile flowFile) {
+        return Pattern.compile(
+                context.getProperty(ATTRIBUTE_REGEX).evaluateAttributeExpressions(flowFile).getValue()
+        );
+    }
+
+    private enum FilterMode {
+        RETAIN,
+        REMOVE,
+    }
+
+    private enum MatchingStrategy {
+        ENUMERATION,
+        REGEX,

Review Comment:
   Recommend using `PATTERN` here instead of `REGEX` to align with Java Pattern object, and adjusted property 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411201844


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);

Review Comment:
   Isn't `List.of` only available for Java 9+? I assumed all language feature used must be available in Java 1.8 as this PR targets NiFi 1.x. 



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on PR #8064:
URL: https://github.com/apache/nifi/pull/8064#issuecomment-1834427491

   Thank you for reviewing and merging #8049, @mosermw.
   
   Can we backport the new processor for the 1.x track as well? I would appreciate a review as you're familiar with the implementation on `main` already. 
   
   As the `main` branch has diverted quite a lot from 1.x already, I backported the changes manually. Besides some changes required for compatibility with Java 1.8, the implementations should align. 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411228506


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   Thanks @EndzeitBegins and @mosermw, that sounds good, I appreciate the response!



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "mosermw (via GitHub)" <gi...@apache.org>.
mosermw commented on PR #8064:
URL: https://github.com/apache/nifi/pull/8064#issuecomment-1834438804

   Yes, sorry, I meant to note that I'm testing this one now.  The code looks good, just doing a final test.


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411247112


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   I've written your suggestions in the Jira ticket, @exceptionfactory. Please let me know if I missed something or you'd like to change something.
   
   Unless communicated otherwise, I will wait for this PR to be merged and start working on NIFI-12446 afterwards, when I can get to it. 
   Or does one of you both want to have a swing at 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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "mosermw (via GitHub)" <gi...@apache.org>.
mosermw closed pull request #8064: NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x)
URL: https://github.com/apache/nifi/pull/8064


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on PR #8064:
URL: https://github.com/apache/nifi/pull/8064#issuecomment-1834442892

   No worries, I just wasn't sure you're aware there is a backport available. Thanks for your work. 


-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411227342


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   That sounds like a reasonable approach. We just have to ensure that the new PRs are merged before the next releases of 1.x and 2.0. 
   
   I opened [NIFI-12446](https://issues.apache.org/jira/browse/NIFI-12446) to track the desired changes.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411195445


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);

Review Comment:
   This should be changed to use `List.of()`



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")

Review Comment:
   The `name` value should be the same as the displayName for new properties, making it easier for programmatic and UI access.
   ```suggestion
               .name("Filter Mode")
               .displayName("Filter Mode")
   ```



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411218921


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   No problem @mosermw, I should have followed up earlier on the pull request for the main branch.
   
   Under the circumstances, it sounds like it may be best to move forward with this pull request after you have completed the review, and then address the naming convention questions in a new Jira issue, with pull requests to both branches.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "mosermw (via GitHub)" <gi...@apache.org>.
mosermw commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411216311


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",

Review Comment:
   @exceptionfactory  I didn't realize you had more comments and I already merged #8049 to main.  We should make sure whatever changes made to the support/nifi-1.x branch in this PR make their way into main, especially any changes to PropertyDescriptors.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411210531


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);

Review Comment:
   That's a good point, my apologies, I was thinking in the context of the main branch. In this case, using Collections.unmodifiableList(Arrays.asList(...))` avoids having to declare the intermediate `_propertyDescriptors`.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")
+            .description("Specifies the strategy to apply on filtered attributes. Either 'Remove' or 'Retain' only the matching attributes.")
+            .required(true)
+            .allowableValues(FILTER_MODE_VALUE_RETAIN, FILTER_MODE_VALUE_REMOVE)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(FILTER_MODE_VALUE_RETAIN.getValue())
+            .build();
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_ENUMERATION = new AllowableValue(
+            "ENUMERATION",
+            "Enumerate attributes",
+            "Provides a set of attribute keys to filter for, separated by a comma delimiter ','."
+    );
+
+    public static final AllowableValue MATCHING_STRATEGY_VALUE_REGEX = new AllowableValue(
+            "REGEX",
+            "Use regular expression",
+            "Provides a regular expression to match keys of attributes to filter for."
+    );
+
+    public static final PropertyDescriptor MATCHING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("MATCHING_STRATEGY")
+            .displayName("Attribute matching strategy")
+            .description("Specifies the strategy to filter attributes by.")
+            .required(true)
+            .allowableValues(MATCHING_STRATEGY_VALUE_ENUMERATION, MATCHING_STRATEGY_VALUE_REGEX)
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .defaultValue(MATCHING_STRATEGY_VALUE_ENUMERATION.getValue())
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_SET = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_SET")
+            .displayName("Set of attributes to filter")
+            .description("A set of attribute names to filter from FlowFiles. Each attribute name is separated by the comma delimiter ','.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_ENUMERATION)
+            .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    public static final PropertyDescriptor ATTRIBUTE_REGEX = new PropertyDescriptor.Builder()
+            .name("ATTRIBUTE_REGEX")
+            .displayName("Regular expression to filter attributes")
+            .description("A regular expression to match names of attributes to filter from FlowFiles.")
+            .required(true)
+            .dependsOn(MATCHING_STRATEGY, MATCHING_STRATEGY_VALUE_REGEX)
+            .addValidator(StandardValidators.REGULAR_EXPRESSION_WITH_EL_VALIDATOR)
+            .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+            .build();
+
+    private final static String DELIMITER_VALUE = ",";
+
+    private final static List<PropertyDescriptor> properties;
+
+    static {
+        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
+        _propertyDescriptors.add(FILTER_MODE);
+        _propertyDescriptors.add(MATCHING_STRATEGY);
+        _propertyDescriptors.add(ATTRIBUTE_SET);
+        _propertyDescriptors.add(ATTRIBUTE_REGEX);
+        properties = Collections.unmodifiableList(_propertyDescriptors);

Review Comment:
   That's a good point, my apologies, I was thinking in the context of the main branch. In this case, using `Collections.unmodifiableList(Arrays.asList(...))` avoids having to declare the intermediate `_propertyDescriptors`.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "EndzeitBegins (via GitHub)" <gi...@apache.org>.
EndzeitBegins commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411204064


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")

Review Comment:
   I'm not against this suggestion, but I think the name should align with the one used for NiFi 2.0 / branch `main`. Thus, we either have to keep this as is or push another commit to `main`, changing the name there as well, before the next release is made, to avoid breaking changes between versions.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


Re: [PR] NIFI-12386 Adds processor FilterAttribute (Java 1.8 / NiFi 1.x) [nifi]

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory commented on code in PR #8064:
URL: https://github.com/apache/nifi/pull/8064#discussion_r1411211679


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/FilterAttribute.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.processors.standard;
+
+import org.apache.nifi.annotation.behavior.DefaultRunDuration;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.AbstractProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSession;
+import org.apache.nifi.processor.Relationship;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.function.Predicate;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+@SideEffectFree
+@SupportsBatching(defaultDuration = DefaultRunDuration.TWENTY_FIVE_MILLIS)
+@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
+@Tags({"attributes", "modification", "filter", "retain", "remove", "delete", "regex", "regular expression", "Attribute Expression Language"})
+@CapabilityDescription("Filters the attributes of a FlowFile by retaining specified attributes and removing the rest or by removing specified attributes and retaining the rest.")
+public class FilterAttribute extends AbstractProcessor {
+
+    public static final Relationship REL_SUCCESS = new Relationship.Builder()
+            .description("All successful FlowFiles are routed to this relationship").name("success").build();
+
+    private final static Set<Relationship> relationships = Collections.singleton(REL_SUCCESS);
+
+
+    public static final AllowableValue FILTER_MODE_VALUE_RETAIN = new AllowableValue(
+            "RETAIN",
+            "Retain",
+            "Retains only the attributes matching the filter, all other attributes are removed."
+    );
+
+    public static final AllowableValue FILTER_MODE_VALUE_REMOVE = new AllowableValue(
+            "REMOVE",
+            "Remove",
+            "Removes the attributes matching the filter, all other attributes are retained."
+    );
+
+    public static final PropertyDescriptor FILTER_MODE = new PropertyDescriptor.Builder()
+            .name("FILTER_MODE")
+            .displayName("Filter mode")

Review Comment:
   That's true, sorry for not flagging this earlier. We should make the change on the main branch as well, then we can make sure both are aligned for the next releases.



-- 
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.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org