You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/12/31 15:12:29 UTC

[GitHub] [nifi] ChrisSamo632 opened a new pull request, #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   # Summary
   
   [NIFI-9206](https://issues.apache.org/jira/browse/NIFI-9206) add the `RemoveRecordField` processor to allow for remove of Record Field(s) from FlowFile content using RecordPath specification similar to `UpdateRecord`.
   
   Caters for simple as well as complex, nested, collection and choice `DataFieldType`s.
   
   A rebase of @pgyori's #5381 taking @tpalfy's comments into account (as well as @Lehel44's comments from #5038).
   
   # 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-9206`
   - [x] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-9206`
   
   ### 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`
     - [ ] JDK 8
     - [ ] JDK 11
     - [x] JDK 17
   
   ### Licensing
   
   - ~[ ] 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)~
   - ~[ ] 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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;

Review Comment:
   These should be changed to JUnit 5 Jupiter.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestRemoveRecordField {
+
+    private TestRunner runner;
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(RemoveRecordField.class);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+        runner.setProperty(jsonWriter, "suppress-nulls", "never-suppress");
+        runner.setProperty(jsonWriter, "Pretty Print JSON", "true");
+        runner.setProperty(jsonWriter, "Schema Write Strategy", "full-schema-attribute");

Review Comment:
   The property names should use the Property Descriptor instead of the string names.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestRemoveRecordField {
+
+    private TestRunner runner;
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(RemoveRecordField.class);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+        runner.setProperty(jsonWriter, "suppress-nulls", "never-suppress");
+        runner.setProperty(jsonWriter, "Pretty Print JSON", "true");
+        runner.setProperty(jsonWriter, "Schema Write Strategy", "full-schema-attribute");
+        runner.enableControllerService(jsonWriter);
+        runner.setProperty(AbstractRecordProcessor.RECORD_WRITER, "writer");
+    }
+
+    @Test
+    public void testNotRootPath() throws InitializationException, IOException {
+        setUpJsonReader(null);
+
+        runner.setProperty("root_path", "/");
+        runner.assertNotValid();
+    }
+
+    @Test
+    public void testNotRootPathEL() throws InitializationException, IOException {
+        // must have at least 1 dynamic property to be valid
+        runner.assertNotValid();
+
+        setUpJsonReader(null);
+
+        runner.setProperty("root_path", "${remove.path}");
+        runner.assertValid();
+
+        runner.enqueue(
+                "{}".getBytes(StandardCharsets.UTF_8),
+                Collections.singletonMap("remove.path", "/")
+        );
+
+        runner.run();
+        runner.assertAllFlowFilesTransferred(AbstractRecordProcessor.REL_FAILURE, 1);
+        runner.getFlowFilesForRelationship(AbstractRecordProcessor.REL_FAILURE).get(0)
+                .assertAttributeEquals("record.error.message", "org.apache.nifi.processor.exception.ProcessException Thrown");
+    }
+
+    @Test
+    public void testRemoveSimpleFieldWithSchemaInference() throws InitializationException, IOException {
+        final String inputContent = "src/test/resources/TestRemoveRecordField/input/complex-person.json";
+        final String outputContent = "src/test/resources/TestRemoveRecordField/output/complex-person-no-dateOfBirth.json";

Review Comment:
   The numerous paths should remove the `src/test/resources/TestRemoveRecordField` repeated prefix using a shared utility method.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RemoveRecordField.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.DynamicProperty;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.record.path.RecordFieldRemover;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.record.Record;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+
+@EventDriven
+@SideEffectFree
+@SupportsBatching
+@InputRequirement(Requirement.INPUT_REQUIRED)
+@Tags({"update", "record", "generic", "schema", "json", "csv", "avro", "freeform", "text", "remove", "delete"})
+@CapabilityDescription("Modifies the contents of a FlowFile that contains Record-oriented data (i.e. data that can be read via a RecordReader and written by a RecordWriter) "
+        + "by removing selected fields. This Processor requires that at least one user-defined Property be added. "
+        + "The name of the property is ignored by the processor, but could be a meaningful identifier for the user. "
+        + "The value of the property should indicate a RecordPath that determines the field to be removed. "
+        + "The processor executes the removal in the order in which these properties are added to the processor. "
+        + "Set the \"Record Writer\" to \"Inherit Record Schema\" in order to use the updated Record Schema modified when removing Fields.")
+@WritesAttributes({
+        @WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.")
+})
+@DynamicProperty(name = "(Ignored)", value = "A RecordPath to the field to be removed.",
+        description = "Allows users to specify fields to remove that match the RecordPath.",
+        expressionLanguageScope = ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+@SeeAlso({UpdateRecord.class})
+public class RemoveRecordField extends AbstractRecordProcessor {
+    private volatile RecordPathCache recordPathCache;
+    private volatile List<RecordFieldRemover.RecordPathRemovalProperties> recordPathsToRemove;
+
+    @Override
+    protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
+        return new PropertyDescriptor.Builder()
+                .name(propertyDescriptorName)
+                .displayName(propertyDescriptorName)
+                .description("The RecordPath to the field that needs to be removed for " + propertyDescriptorName)
+                .required(false)
+                .dynamic(true)
+                .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
+                .addValidator(new RecordPathValidator())
+                .build();
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(final ValidationContext validationContext) {
+        final boolean containsDynamic = validationContext.getProperties().keySet().stream().anyMatch(PropertyDescriptor::isDynamic);
+
+        if (containsDynamic) {
+            final List<ValidationResult> validationResults = new ArrayList<>(validationContext.getProperties().size());
+            validationContext.getProperties().keySet().stream().filter(PropertyDescriptor::isDynamic)
+                    .forEach(property -> {
+                        final String path = validationContext.getProperty(property).evaluateAttributeExpressions().getValue();
+                        if ("/".equals(path)) {

Review Comment:
   Recommend defining a static member variable named `ROOT_PATH` for `/` and reusing here in the process method.



##########
nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java:
##########
@@ -62,47 +63,166 @@ public void testPreventsTwoFieldsWithConflictingNamesAliases() {
     }
 
     @Test
-    public void testHashCodeAndEqualsWithSelfReferencingSchema() {
+    void testHashCodeAndEqualsWithSelfReferencingSchema() {
         final SimpleRecordSchema schema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
 
         final List<RecordField> personFields = new ArrayList<>();
         personFields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
         personFields.add(new RecordField("sibling", RecordFieldType.RECORD.getRecordDataType(schema)));
 
         schema.setFields(personFields);
-
-        schema.hashCode();
-        assertTrue(schema.equals(schema));
+        assertEquals(schema, schema);
 
         final SimpleRecordSchema secondSchema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
         secondSchema.setFields(personFields);
-        assertTrue(schema.equals(secondSchema));
-        assertTrue(secondSchema.equals(schema));
+        assertEquals(schema.hashCode(), secondSchema.hashCode());
+        assertEquals(schema, secondSchema);
+        assertEquals(secondSchema, schema);
+    }
+
+    @Test
+    void testEqualsSimpleSchema() {
+        // GIVEN

Review Comment:
   The `GIVEN/WHEN/THEN` comments should be removed.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;

Review Comment:
   Recommend replacing Hamcrest assertions with JUnit 5 assertions.



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RemoveRecordField.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * 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.DynamicProperty;
+import org.apache.nifi.annotation.behavior.EventDriven;
+import org.apache.nifi.annotation.behavior.InputRequirement;
+import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
+import org.apache.nifi.annotation.behavior.SideEffectFree;
+import org.apache.nifi.annotation.behavior.SupportsBatching;
+import org.apache.nifi.annotation.behavior.WritesAttribute;
+import org.apache.nifi.annotation.behavior.WritesAttributes;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.SeeAlso;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnScheduled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.record.path.RecordFieldRemover;
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.record.path.validation.RecordPathValidator;
+import org.apache.nifi.serialization.record.Record;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+
+
+@EventDriven

Review Comment:
   This annotation should be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [nifi] ChrisSamo632 commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   @exceptionfactory I see a change to the `accumulo` bundle's `KeyScehama`, which extends `RecordSchema` and so needed changing to implement the updated `RecordSchema` API; and a change to a unit test for an `asn1` test case that (presumably) didn't pass without the modifications.
   
   Otherwise this PR includes the addition of the new `RemoveRecordField` processor and required changes to the associated `Record` and `RecordPath` framework classes (which obviously could impact a lot).
   
   I'm not clear on what
   > number changes that apply to other Processors. Although they are useful in themselves, they should be split off to separate PRs where possible
   
   can be split off from this into separate PRs
   
   As to
   > This new PR also includes a very large number of test files. Do you think all of those are necessary? The size and scope seem larger than needed at a glance, but I have not yet given the tests a thorough review.
   
   I agree, I was surprised there were so many test files ubt presumably these were included to try and cover the wide range of possible Record modifications that one could perform with the new processor (I largely just tried to re-invigorate #5381, I probably didn't check through every new test case/file in detail when doing so, I'll be honest)


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


[GitHub] [nifi] ChrisSamo632 commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   > @ChrisSamo632 you have a merge conflict.
   
   Thanks for the heads-up @MikeThomsen, recent removals of deprecated modules had meant the changes to the RAT exclusion list in the `nifi-standard-processors` module's `pom.xml` file were confused - should now be resolved


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


[GitHub] [nifi] exceptionfactory closed pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

Posted by "exceptionfactory (via GitHub)" <gi...@apache.org>.
exceptionfactory closed pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records
URL: https://github.com/apache/nifi/pull/6816


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


[GitHub] [nifi] exceptionfactory commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   Thanks for the summary @ChrisSamo632.
   
   Looking over the current PR, there are a number changes that apply to other Processors. Although they are useful in themselves, they should be split off to separate PRs where possible.
   
   This new PR also includes a very large number of test files. Do you think all of those are necessary? The size and scope seem larger than needed at a glance, but I have not yet given the tests a thorough review.


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


[GitHub] [nifi] exceptionfactory commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   Thanks for the reply and summary @ChrisSamo632. On further review, the changes to those other components make sense together with the adjustments necessary for this PR, so breaking things up is not necessary.
   
   I think it would be worthwhile to review the scope of the test files and reduce the number of test files if sufficient coverage can be achieved.


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


[GitHub] [nifi] MikeThomsen commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   @ChrisSamo632 you have a merge conflict.


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


[GitHub] [nifi] ChrisSamo632 commented on a diff in pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestRemoveRecordField {
+
+    private TestRunner runner;
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(RemoveRecordField.class);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+        runner.setProperty(jsonWriter, "suppress-nulls", "never-suppress");
+        runner.setProperty(jsonWriter, "Pretty Print JSON", "true");
+        runner.setProperty(jsonWriter, "Schema Write Strategy", "full-schema-attribute");

Review Comment:
   Think I tried using those but the PropertyDescriptors are package private (unless I'm mis-recalling) - I can make them public, but was resisting more changes to modules, although I was surprised these properties weren't already public to be honest - happy to make the change



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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

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


[GitHub] [nifi] ChrisSamo632 commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   @exceptionfactory addressed your comments, ready for re-review


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


[GitHub] [nifi] ChrisSamo632 commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   If you know all of the fields in your data and are happy/able to write them all into a `SELECT` statement, omitting just those that aren't wanted, then yes `QueryRecord` can be a useful alternative here.
   
   Where I'd have liked this option previosuly is when we receive different formats of data from different sources although each of those contains a set of known fields, they contain some that aren't wanted but also some we don't know about beforehand (e.g. "extra" columns that we need to retain but aren't in a set of "known" fields that we could include in a `SELECT`). There's not a way with `QueryRecord` (that I can immediately think of) to say "select these 5 fields, omit these 3, retain any other fields", but a way of deleting just those 3 fields from Records would allow this to happen - instead I've ended up writing Groovy Scripts (or similar) to cater for this previously and scripting often seems like a way that people don't want to go.
   
   I could see this kind of functionality being useful alongside that already present in `UpdateRecord` and it seems this function originally was being implemented for that processor, but the idea of a separate processor was ultimately preferred (looking back at Jiras and previous PRs)


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


[GitHub] [nifi] exceptionfactory commented on a diff in pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java:
##########
@@ -0,0 +1,396 @@
+/*
+ * 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 com.fasterxml.jackson.databind.JsonNode;
+import com.fasterxml.jackson.databind.ObjectMapper;
+import org.apache.nifi.json.JsonRecordSetWriter;
+import org.apache.nifi.json.JsonTreeReader;
+import org.apache.nifi.reporting.InitializationException;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
+import org.apache.nifi.schema.inference.SchemaInferenceUtil;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+public class TestRemoveRecordField {
+
+    private TestRunner runner;
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    @Before
+    public void setup() throws InitializationException {
+        runner = TestRunners.newTestRunner(RemoveRecordField.class);
+
+        final JsonRecordSetWriter jsonWriter = new JsonRecordSetWriter();
+        runner.addControllerService("writer", jsonWriter);
+        runner.setProperty(jsonWriter, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.INHERIT_RECORD_SCHEMA);
+        runner.setProperty(jsonWriter, "suppress-nulls", "never-suppress");
+        runner.setProperty(jsonWriter, "Pretty Print JSON", "true");
+        runner.setProperty(jsonWriter, "Schema Write Strategy", "full-schema-attribute");

Review Comment:
   Ahh, that makes sense, thanks for the background. Seems like a reasonable change, although that raises the question as to whether this test should use a Mock Writer instead of the JSON Writer. Making the adjustment to visibility seems good enough for now.



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


[GitHub] [nifi] ChrisSamo632 commented on a diff in pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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


##########
nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java:
##########
@@ -62,47 +63,166 @@ public void testPreventsTwoFieldsWithConflictingNamesAliases() {
     }
 
     @Test
-    public void testHashCodeAndEqualsWithSelfReferencingSchema() {
+    void testHashCodeAndEqualsWithSelfReferencingSchema() {
         final SimpleRecordSchema schema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
 
         final List<RecordField> personFields = new ArrayList<>();
         personFields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
         personFields.add(new RecordField("sibling", RecordFieldType.RECORD.getRecordDataType(schema)));
 
         schema.setFields(personFields);
-
-        schema.hashCode();
-        assertTrue(schema.equals(schema));
+        assertEquals(schema, schema);
 
         final SimpleRecordSchema secondSchema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
         secondSchema.setFields(personFields);
-        assertTrue(schema.equals(secondSchema));
-        assertTrue(secondSchema.equals(schema));
+        assertEquals(schema.hashCode(), secondSchema.hashCode());
+        assertEquals(schema, secondSchema);
+        assertEquals(secondSchema, schema);
+    }
+
+    @Test
+    void testEqualsSimpleSchema() {
+        // GIVEN

Review Comment:
   Thought I'd removed all of those, good catch



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


[GitHub] [nifi] ChrisSamo632 commented on pull request #6816: NIFI-9206: Add RemoveRecordField processor and implement the ability to remove fields from records

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

   @exceptionfactory I've reviewed the tests and removed a few as they were overlapping enough to (I think) de-duplicate.
   
   However, there are still quite a lot being added, but I think they're justified because they cover a combination of different Schema Data Field Types (e.g. Scalar, Array, Map, Choice) along with top-level and nested fields being removed.
   
   I had a bit of a play with trying to avoid the need for both input and output schema files, but it was turning into relatively unmanageable code, e.g. to read the input schema into a JsonNode, then try to remove the fields we'd expect to not exist; so I think having all the inputs/outputs defined as files makes sense, although I've used schema inference in a couple of places to at least show that's possible too.


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