You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by ex...@apache.org on 2023/06/26 15:32:10 UTC

[nifi] branch main updated: NIFI-9206 Added RemoveRecordField Processor

This is an automated email from the ASF dual-hosted git repository.

exceptionfactory pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/nifi.git


The following commit(s) were added to refs/heads/main by this push:
     new 3f5ed23504 NIFI-9206 Added RemoveRecordField Processor
3f5ed23504 is described below

commit 3f5ed2350476ea258b0d96b00987e5deb626af53
Author: Chris Sampson <ch...@gmail.com>
AuthorDate: Sat Dec 31 15:08:02 2022 +0000

    NIFI-9206 Added RemoveRecordField Processor
    
    This closes #6816
    
    Co-authored-by: Peter Gyori <pe...@gmail.com>
    Co-authored-by: Chris Sampson <ch...@gmail.com>
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi/record/path/ArrayIndexFieldValue.java     |  14 +-
 .../org/apache/nifi/record/path/FieldValue.java    |  10 +
 .../nifi/record/path/MapEntryFieldValue.java       |  15 +-
 .../nifi/record/path/RecordFieldRemover.java       | 138 +++++
 .../nifi/record/path/StandardFieldValue.java       |  23 +
 .../nifi/record/path/paths/ChildFieldPath.java     |  27 +-
 .../nifi/record/path/TestRecordFieldRemover.java   |  84 +++
 .../nifi/serialization/SimpleRecordSchema.java     |  60 +-
 .../apache/nifi/serialization/record/DataType.java |  10 +
 .../nifi/serialization/record/MapRecord.java       |  28 +-
 .../apache/nifi/serialization/record/Record.java   |  16 +
 .../record/RecordFieldRemovalPath.java             |  49 ++
 .../nifi/serialization/record/RecordSchema.java    |  22 +
 .../serialization/record/type/ArrayDataType.java   |  17 +
 .../serialization/record/type/ChoiceDataType.java  |  19 +
 .../serialization/record/type/MapDataType.java     |  17 +
 .../serialization/record/type/RecordDataType.java  |  27 +
 .../nifi/serialization/TestSimpleRecordSchema.java | 166 +++++-
 .../org/apache/nifi/accumulo/data/KeySchema.java   |  17 +
 .../TestJASN1RecordReaderWithComplexTypes.java     |  16 +-
 .../SchemaRegistryRecordSetWriter.java             |  16 +-
 .../nifi-standard-processors/pom.xml               | 321 ++++++-----
 .../nifi/processors/standard/ForkRecord.java       |  21 +-
 .../processors/standard/RemoveRecordField.java     | 138 +++++
 .../services/org.apache.nifi.processor.Processor   |   1 +
 .../additionalDetails.html                         | 633 +++++++++++++++++++++
 .../processors/standard/TestRemoveRecordField.java | 399 +++++++++++++
 .../TestRemoveRecordField/input/address-array.json |  30 +
 .../input/address-choice.json                      |  25 +
 .../TestRemoveRecordField/input/bookshelf-map.json |  16 +
 .../input/complex-person.json                      |  29 +
 .../TestRemoveRecordField/input/name-choice.json   |  10 +
 .../input_schema/address-array.avsc                |  52 ++
 .../input_schema/address-choice.avsc               |  51 ++
 .../input_schema/bookshelf-map.avsc                |  27 +
 .../input_schema/complex-person.avsc               |   1 +
 .../input_schema/name-choice.avsc                  |  31 +
 .../output/address-array-empty.json                |   5 +
 .../output/address-array-one-element-removed.json  |  19 +
 .../output/address-array-removed.json              |   4 +
 ...ddress-array-zip-removed-from-all-elements.json |  23 +
 ...address-array-zip-removed-from-one-element.json |  26 +
 ...ddress-choice-letter-removed-from-building.json |  20 +
 .../output/bookshelf-map-all-items-removed.json    |   3 +
 ...kshelf-map-field-removed-from-all-elements.json |  13 +
 ...okshelf-map-field-removed-from-one-element.json |  16 +
 .../output/bookshelf-map-one-item-removed.json     |  12 +
 .../complex-person-multiple-fields-removed.json    |  23 +
 .../output/complex-person-no-dateOfBirth.json      |  29 +
 ...plex-person-no-workAddress-building-letter.json |  30 +
 .../output/complex-person-no-workAddress.json      |  22 +
 .../output/complex-person.json                     |  31 +
 .../output/name-choice-firstName-removed.json      |   9 +
 .../output_schema/address-array-field-removed.avsc |   1 +
 .../output_schema/address-array-zip-removed.avsc   |   1 +
 .../output_schema/address-array.avsc               |   1 +
 ...ddress-choice-letter-removed-from-building.avsc |   1 +
 .../bookshelf-map-author-field-removed.avsc        |   1 +
 .../output_schema/bookshelf-map.avsc               |   1 +
 .../complex-person-multiple-fields-removed.avsc    |   1 +
 .../complex-person-no-dateOfBirth.avsc             |   1 +
 ...plex-person-no-workAddress-building-letter.avsc |   1 +
 .../complex-person-no-workAddress.avsc             |   1 +
 .../output_schema/complex-person.avsc              |   1 +
 .../name-choice-firstName-removed.avsc             |   1 +
 .../org/apache/nifi/json/JsonRecordSetWriter.java  |  18 +-
 66 files changed, 2667 insertions(+), 224 deletions(-)

diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
index b69fbce458..04b11b3a56 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/ArrayIndexFieldValue.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.record.path;
 
+import org.apache.commons.lang3.ArrayUtils;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
@@ -51,12 +52,21 @@ public class ArrayIndexFieldValue extends StandardFieldValue {
 
     @Override
     public void updateValue(final Object newValue) {
-        getParentRecord().get().setArrayValue(getField().getFieldName(), getArrayIndex(), newValue);
+        getParentRecord().ifPresent(parent -> parent.setArrayValue(getField().getFieldName(), getArrayIndex(), newValue));
     }
 
     @Override
     public void updateValue(final Object newValue, final DataType dataType) {
-        getParentRecord().get().setArrayValue(getField().getFieldName(), getArrayIndex(), newValue);
+        getParentRecord().ifPresent(parent -> parent.setArrayValue(getField().getFieldName(), getArrayIndex(), newValue));
+    }
+
+    @Override
+    public void remove() {
+        getParent().ifPresent(parent -> {
+            if (parent.getValue() instanceof Object[]) {
+                parent.updateValue(ArrayUtils.remove((Object[]) parent.getValue(), index));
+            }
+        });
     }
 
     @Override
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
index 084ce7376f..c0807fb37d 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/FieldValue.java
@@ -62,4 +62,14 @@ public interface FieldValue {
      * @param dataType the data type to use if the Record's schema does not already include this field
      */
     void updateValue(Object newValue, DataType dataType);
+
+    /**
+     * Removes this FieldValue instance from its parent FieldValue instance
+     */
+    void remove();
+
+    /**
+     * Removes the content of this FieldValue instance
+     */
+    void removeContent();
 }
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
index f28064de19..54d4ab2ffe 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/MapEntryFieldValue.java
@@ -20,6 +20,7 @@ package org.apache.nifi.record.path;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 
+import java.util.Map;
 import java.util.Objects;
 
 public class MapEntryFieldValue extends StandardFieldValue {
@@ -36,12 +37,22 @@ public class MapEntryFieldValue extends StandardFieldValue {
 
     @Override
     public void updateValue(final Object newValue) {
-        getParentRecord().get().setMapValue(getField().getFieldName(), getMapKey(), newValue);
+        getParentRecord().ifPresent(parent -> parent.setMapValue(getField().getFieldName(), getMapKey(), newValue));
     }
 
     @Override
     public void updateValue(final Object newValue, final DataType dataType) {
-        getParentRecord().get().setMapValue(getField().getFieldName(), getMapKey(), newValue);
+        getParentRecord().ifPresent(parent -> parent.setMapValue(getField().getFieldName(), getMapKey(), newValue));
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public void remove() {
+        getParent().ifPresent(parent -> {
+            if (parent.getValue() instanceof Map) {
+                ((Map<String, Object>) parent.getValue()).remove(mapKey);
+            }
+        });
     }
 
     @Override
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/RecordFieldRemover.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/RecordFieldRemover.java
new file mode 100644
index 0000000000..579e14c01a
--- /dev/null
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/RecordFieldRemover.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.record.path;
+
+import org.apache.nifi.record.path.util.RecordPathCache;
+import org.apache.nifi.serialization.record.Record;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
+import org.apache.nifi.serialization.record.RecordSchema;
+
+import java.util.List;
+import java.util.Optional;
+import java.util.regex.Pattern;
+import java.util.stream.Collectors;
+
+public class RecordFieldRemover {
+    private final RecordPathCache recordPathCache;
+    private final Record record;
+
+    private boolean fieldsChanged;
+
+    public RecordFieldRemover(final Record record, final RecordPathCache recordPathCache) {
+        this.record = record;
+        this.recordPathCache = recordPathCache;
+    }
+
+    public Record getRecord() {
+        if (fieldsChanged) {
+            record.regenerateSchema();
+        }
+        return record;
+    }
+
+    public void remove(final RecordPathRemovalProperties recordPathRemovalProperties) {
+        final RecordPath recordPath = recordPathCache.getCompiled(recordPathRemovalProperties.getRecordPath());
+        final RecordPathResult recordPathResult = recordPath.evaluate(record);
+        final List<FieldValue> selectedFields = recordPathResult.getSelectedFields().collect(Collectors.toList());
+
+        if (!selectedFields.isEmpty()) {
+            if (recordPathRemovalProperties.isAppliedToAllElementsInCollection()) {
+                // all elements have the same parent, so navigate up from the first element in the collection
+                selectedFields.get(0).getParent().ifPresent(FieldValue::removeContent);
+            } else {
+                selectedFields.forEach(FieldValue::remove);
+            }
+
+            if (recordPathRemovalProperties.isRemovingFieldsNotJustElementsFromWithinCollection()) {
+                removeFieldsFromSchema(selectedFields);
+            }
+
+            fieldsChanged = true;
+        }
+    }
+
+    private void removeFieldsFromSchema(final List<FieldValue> selectedFields) {
+        final List<RecordFieldRemovalPath> paths = getConcretePaths(selectedFields);
+        final RecordSchema schema = record.getSchema();
+        paths.forEach(schema::removePath);
+    }
+
+    private List<RecordFieldRemovalPath> getConcretePaths(final List<FieldValue> selectedFields) {
+        return selectedFields.stream().map(field -> {
+            final RecordFieldRemovalPath path = new RecordFieldRemovalPath();
+            addToPathIfNotRoot(field, path);
+
+            Optional<FieldValue> parentOptional = field.getParent();
+            while (parentOptional.isPresent()) {
+                final FieldValue parent = parentOptional.get();
+                addToPathIfNotRoot(parent, path);
+                parentOptional = parent.getParent();
+            }
+            return path;
+        }).collect(Collectors.toList());
+    }
+
+    private void addToPathIfNotRoot(final FieldValue field, final RecordFieldRemovalPath path) {
+        field.getParent().ifPresent(parent -> path.add(field.getField().getFieldName()));
+    }
+
+    public static class RecordPathRemovalProperties {
+        private static final Pattern ALL_ELEMENTS_REGEX = Pattern.compile(".*\\[\\s*(?:\\*|0\\s*\\.\\.\\s*-1)\\s*]$");
+        private static final Pattern ARRAY_ELEMENTS_REGEX = Pattern.compile("\\[\\s*-?\\d+(?:\\s*,\\s*-?\\d+)*+\\s*]");
+        private static final Pattern MAP_ELEMENTS_REGEX = Pattern.compile("\\[\\s*'[^']+'(?:\\s*,\\s*'[^']+')*+\\s*]");
+
+        private final String recordPath;
+
+        private final boolean appliedToAllElementsInCollection;
+        private final boolean appliedToIndividualArrayElements;
+        private final boolean appliedToIndividualMapElements;
+
+        public RecordPathRemovalProperties(final String recordPath) {
+            this.recordPath = recordPath;
+
+            // ends with [*] or [0..-1]
+            this.appliedToAllElementsInCollection = ALL_ELEMENTS_REGEX.matcher(recordPath).matches();
+
+            // contains an array reference [] with one or more element references, e.g. [1], [ 1, -1]
+            this.appliedToIndividualArrayElements = ARRAY_ELEMENTS_REGEX.matcher(recordPath).find();
+
+            // contains a map reference [] with one or more element references, e.g. ['one'], ['one' , 'two' ]
+            this.appliedToIndividualMapElements = MAP_ELEMENTS_REGEX.matcher(recordPath).find();
+        }
+
+        String getRecordPath() {
+            return recordPath;
+        }
+
+        boolean isAppliedToAllElementsInCollection() {
+            return appliedToAllElementsInCollection;
+        }
+
+        boolean isAppliedToIndividualArrayElements() {
+            return appliedToIndividualArrayElements;
+        }
+
+        boolean isAppliedToIndividualMapElements() {
+            return appliedToIndividualMapElements;
+        }
+
+        boolean isRemovingFieldsNotJustElementsFromWithinCollection() {
+            return !isAppliedToIndividualArrayElements() && !isAppliedToIndividualMapElements();
+        }
+    }
+}
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
index 8ab03ca373..0d81e86854 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/StandardFieldValue.java
@@ -21,8 +21,10 @@ import org.apache.nifi.record.path.util.Filters;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldType;
 
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.Objects;
 import java.util.Optional;
 
@@ -128,6 +130,27 @@ public class StandardFieldValue implements FieldValue {
         updateValue(newValue, recordField);
     }
 
+    @Override
+    public void remove() {
+        getParent().ifPresent(p -> {
+            if (Filters.isRecord(p)) {
+                final Record parentRecord = (Record) p.getValue();
+                if (parentRecord != null) {
+                    parentRecord.remove(field);
+                }
+            }
+        });
+    }
+
+    @Override
+    public void removeContent() {
+        if (field.getDataType().getFieldType() == RecordFieldType.ARRAY) {
+            updateValue(new Object[0]);
+        } else if (field.getDataType().getFieldType() == RecordFieldType.MAP) {
+            updateValue(Collections.emptyMap());
+        }
+    }
+
     private void updateValue(final Object newValue, final RecordField field) {
         final Optional<Record> parentRecord = getParentRecord();
         if (!parentRecord.isPresent()) {
diff --git a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/paths/ChildFieldPath.java b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/paths/ChildFieldPath.java
index ac0385a5fc..97196e9e4c 100644
--- a/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/paths/ChildFieldPath.java
+++ b/nifi-commons/nifi-record-path/src/main/java/org/apache/nifi/record/path/paths/ChildFieldPath.java
@@ -17,9 +17,6 @@
 
 package org.apache.nifi.record.path.paths;
 
-import java.util.Optional;
-import java.util.stream.Stream;
-
 import org.apache.nifi.record.path.FieldValue;
 import org.apache.nifi.record.path.RecordPathEvaluationContext;
 import org.apache.nifi.record.path.StandardFieldValue;
@@ -27,6 +24,10 @@ import org.apache.nifi.record.path.util.Filters;
 import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.type.RecordDataType;
+
+import java.util.Optional;
+import java.util.stream.Stream;
 
 public class ChildFieldPath extends RecordPathSegment {
     private final String childName;
@@ -41,19 +42,31 @@ public class ChildFieldPath extends RecordPathSegment {
         return new StandardFieldValue(null, field, parent);
     }
 
+    private FieldValue missingChild(final FieldValue parent, final RecordField childField) {
+        final RecordField field = new RecordField(childName, childField.getDataType(), childField.isNullable());
+        return new StandardFieldValue(null, field, parent);
+    }
+
     private FieldValue getChild(final FieldValue fieldValue) {
         if (!Filters.isRecord(fieldValue)) {
             return missingChild(fieldValue);
         }
 
         final Record record = (Record) fieldValue.getValue();
-        if(record == null) {
-            return missingChild(fieldValue);
+        if (record == null) {
+            final RecordField parent = fieldValue.getField();
+            if (parent != null && parent.getDataType() instanceof RecordDataType) {
+                final Optional<RecordField> childFieldOptional = ((RecordDataType) parent.getDataType()).getChildSchema().getField(childName);
+                return childFieldOptional.map(recordField -> missingChild(fieldValue, recordField)).orElseGet(() -> missingChild(fieldValue));
+            } else {
+                return missingChild(fieldValue);
+            }
         }
 
         final Object value = record.getValue(childName);
         if (value == null) {
-            return missingChild(fieldValue);
+            final Optional<RecordField> childFieldOptional = record.getSchema().getField(childName);
+            return childFieldOptional.map(recordField -> missingChild(fieldValue, recordField)).orElseGet(() -> missingChild(fieldValue));
         }
 
         final Optional<RecordField> field = record.getSchema().getField(childName);
@@ -68,6 +81,6 @@ public class ChildFieldPath extends RecordPathSegment {
     public Stream<FieldValue> evaluate(final RecordPathEvaluationContext context) {
         return getParentPath().evaluate(context)
             // map to Optional<FieldValue> containing child element
-            .map(fieldVal -> getChild(fieldVal));
+            .map(this::getChild);
     }
 }
diff --git a/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordFieldRemover.java b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordFieldRemover.java
new file mode 100644
index 0000000000..c293bc1dc2
--- /dev/null
+++ b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordFieldRemover.java
@@ -0,0 +1,84 @@
+/*
+ * 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.record.path;
+
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+class TestRecordFieldRemover {
+    @ParameterizedTest
+    @ValueSource(strings = {"[*]", "[ * ]", "[* ]", "[ *]", "/street[*]", "//[*]", //
+            "[0..-1]", "[ 0..-1]", "[0..-1 ]", "[0 ..-1]", "[0.. -1]", "[ 0 .. -1 ]"})
+    void testIsAppliedToAllElementsInCollection(final String input) {
+        assertTrue(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToAllElementsInCollection());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[]", "", "/", "//", "/street", "[*]/street", "[1]", "[ -1 ]", "[0, 1]", "['one']"})
+    void testNotIsAppliedToAllElementsInCollection(final String input) {
+        assertFalse(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToAllElementsInCollection());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[1]", "[-1]", "[1]/street", "/street[ 1, 2 ]", "//[ -1,-2,3]"})
+    void testIsAppliedToIndividualArrayElements(final String input) {
+        assertTrue(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToIndividualArrayElements());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[]", "", "/", "//", "/street", "[*]", "[ 0..-1 ]", "['one']"})
+    void testNotIsAppliedToIndividualArrayElements(final String input) {
+        assertFalse(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToIndividualArrayElements());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"['one']", "[ 'one', 'two' ]", "['one']/street", "/street[ 'one, two' ]", "//['one' , 'two']"})
+    void testIsAppliedToIndividualMapElements(final String input) {
+        assertTrue(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToIndividualMapElements());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[]", "", "/", "//", "/street", "[1]", "[ -1 ]", "[0, 1]", "[*]", "[ 0..-1 ]"})
+    void testNotIsAppliedToIndividualMapElements(final String input) {
+        assertFalse(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isAppliedToIndividualMapElements());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[]", "", "/", "//", "/street", "[*]", "[ 0..-1 ]"})
+    void testIsPathRemovalRequiresSchemaModification(final String input) {
+        assertTrue(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isRemovingFieldsNotJustElementsFromWithinCollection());
+    }
+
+    @ParameterizedTest
+    @ValueSource(strings = {"[1]", "[-1]", "/street[ 1, 2 ]", "//[ -1,-2,3]", //
+            "['one']", "[ 'one', 'two' ]", "/street[ 'one, two' ]", "//['one' , 'two']"})
+    void testNotIsPathRemovalRequiresSchemaModification(final String input) {
+        assertFalse(new RecordFieldRemover.RecordPathRemovalProperties("/addresses" + input)
+                .isRemovingFieldsNotJustElementsFromWithinCollection());
+    }
+}
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
index 9a48b02cc0..88f31c6cc0 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java
@@ -19,6 +19,7 @@ package org.apache.nifi.serialization;
 
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.SchemaIdentifier;
 
@@ -34,10 +35,10 @@ import java.util.stream.Collectors;
 public class SimpleRecordSchema implements RecordSchema {
     private List<RecordField> fields = null;
     private Map<String, RecordField> fieldMap = null;
-    private final boolean textAvailable;
+    private boolean textAvailable;
     private final AtomicReference<String> text = new AtomicReference<>();
-    private final String schemaFormat;
-    private final SchemaIdentifier schemaIdentifier;
+    private String schemaFormat;
+    private SchemaIdentifier schemaIdentifier;
     private String schemaName;
     private String schemaNamespace;
     private volatile int hashCode;
@@ -167,12 +168,20 @@ public class SimpleRecordSchema implements RecordSchema {
         }
 
         final RecordSchema other = (RecordSchema) obj;
-        if (getSchemaNamespace().isPresent() && getSchemaNamespace().equals(other.getSchemaNamespace())
-                && getSchemaName().isPresent() && getSchemaName().equals(other.getSchemaName())) {
-            return true;
+        final boolean thisIsRecursive = isRecursive();
+        final boolean otherIsRecursive = other.isRecursive();
+        if (thisIsRecursive && otherIsRecursive) {
+            if (getSchemaNamespace().isPresent() && getSchemaNamespace().equals(other.getSchemaNamespace())
+                    && getSchemaName().isPresent() && getSchemaName().equals(other.getSchemaName())) {
+                return true;
+            } else {
+                return fields.equals(other.getFields());
+            }
+        } else if (thisIsRecursive || otherIsRecursive) {
+            return false;
+        } else {
+            return fields.equals(other.getFields());
         }
-
-        return fields.equals(other.getFields());
     }
 
     @Override
@@ -231,7 +240,7 @@ public class SimpleRecordSchema implements RecordSchema {
      * Set schema name.
      * @param schemaName schema name as defined in a root record.
      */
-    public void setSchemaName(String schemaName) {
+    public void setSchemaName(final String schemaName) {
         this.schemaName = schemaName;
     }
 
@@ -244,7 +253,7 @@ public class SimpleRecordSchema implements RecordSchema {
      * Set schema namespace.
      * @param schemaNamespace schema namespace as defined in a root record.
      */
-    public void setSchemaNamespace(String schemaNamespace) {
+    public void setSchemaNamespace(final String schemaNamespace) {
         this.schemaNamespace = schemaNamespace;
     }
 
@@ -252,4 +261,35 @@ public class SimpleRecordSchema implements RecordSchema {
     public Optional<String> getSchemaNamespace() {
         return Optional.ofNullable(schemaNamespace);
     }
+
+    @Override
+    public void removeField(final String fieldName) {
+        final List<RecordField> remainingFields = fields.stream()
+                .filter(field -> !field.getFieldName().equals(fieldName)).collect(Collectors.toList());
+
+        if (remainingFields.size() != fields.size()) {
+            fields = null;
+            setFields(remainingFields);
+            text.set(createText(fields));
+            textAvailable = true;
+            schemaFormat = null;
+            schemaIdentifier = SchemaIdentifier.EMPTY;
+            hashCode = 0; // set to 0 to trigger re-calculation
+            hashCode = hashCode();
+        }
+    }
+
+    @Override
+    public void removePath(final RecordFieldRemovalPath path) {
+        if (path.length() == 1) {
+            removeField(path.head());
+        } else if (path.length() != 0) {
+            getField(path.head()).ifPresent(field -> field.getDataType().removePath(path.tail()));
+        }
+    }
+
+    @Override
+    public boolean isRecursive() {
+        return getFields().stream().anyMatch(field -> field.getDataType().isRecursive(Collections.singletonList(this)));
+    }
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/DataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/DataType.java
index 6ed4bd6f88..78772d2cf7 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/DataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/DataType.java
@@ -17,6 +17,7 @@
 
 package org.apache.nifi.serialization.record;
 
+import java.util.List;
 import java.util.Objects;
 
 public class DataType {
@@ -36,6 +37,15 @@ public class DataType {
         return fieldType;
     }
 
+    public void removePath(final RecordFieldRemovalPath path) {
+        // intentionally blank - to be overridden by concrete DataTypes
+    }
+
+    @SuppressWarnings("unused")
+    public boolean isRecursive(final List<RecordSchema> schemas) {
+        return false;
+    }
+
     @Override
     public int hashCode() {
         return 31 + 41 * getFieldType().hashCode() + 41 * (getFormat() == null ? 0 : getFormat().hashCode());
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
index 58ce590551..cf48c0e06a 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/MapRecord.java
@@ -400,6 +400,31 @@ public class MapRecord implements Record {
         }
     }
 
+    @Override
+    public void remove(final RecordField field) {
+        final Optional<RecordField> existingField = resolveField(field);
+        existingField.ifPresent(recordField -> values.remove(recordField.getFieldName()));
+    }
+
+    @Override
+    public void regenerateSchema() {
+        final List<RecordField> schemaFields = new ArrayList<>(schema.getFieldCount());
+
+        for (final RecordField schemaField : schema.getFields()) {
+            final Object fieldValue = getValue(schemaField);
+            if (schemaField.getDataType().getFieldType() == RecordFieldType.CHOICE) {
+                schemaFields.add(schemaField);
+            } else if (fieldValue instanceof Record) {
+                final Record childRecord = (Record) fieldValue;
+                schemaFields.add(new RecordField(schemaField.getFieldName(), RecordFieldType.RECORD.getRecordDataType(childRecord.getSchema()), schemaField.isNullable()));
+            } else {
+                schemaFields.add(schemaField);
+            }
+        }
+
+        schema = new SimpleRecordSchema(schemaFields);
+    }
+
     @Override
     public void setValue(final String fieldName, final Object value) {
         final Optional<RecordField> existingField = setValueAndGetField(fieldName, value);
@@ -518,9 +543,6 @@ public class MapRecord implements Record {
         this.schema = DataTypeUtils.merge(this.schema, other);
     }
 
-
-
-
     @Override
     public void incorporateInactiveFields() {
         final List<RecordField> updatedFields = new ArrayList<>();
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
index 776084ba18..053417864b 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/Record.java
@@ -142,6 +142,22 @@ public interface Record {
      */
     void setValue(RecordField field, Object value);
 
+    /**
+     * Removes the value of a given field from the Record.
+     * It only removes the value of that field but does not modify the schema.
+     * To remove a field completely (from the schema as well as the data)
+     * the {@see org.apache.nifi.record.path.RecordFieldRemover} class should be used.
+     *
+     * @param field the field that should be removed from the record
+     */
+    void remove(RecordField field);
+
+    /**
+     * Creates a new schema for the Record based on the Record's field types.
+     * In case any of the Record's fields were changed, this method propagates the changes to the parent Record.
+     */
+    void regenerateSchema();
+
     /**
      * Updates the value of a the specified index of a field. If the field specified
      * is not present in this Record's schema, this method will do nothing. If the field specified
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldRemovalPath.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldRemovalPath.java
new file mode 100644
index 0000000000..51f97b143a
--- /dev/null
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordFieldRemovalPath.java
@@ -0,0 +1,49 @@
+/*
+ * 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.serialization.record;
+
+import java.util.ArrayList;
+import java.util.List;
+
+public class RecordFieldRemovalPath {
+    private final List<String> path;
+
+    public RecordFieldRemovalPath() {
+        path = new ArrayList<>();
+    }
+
+    private RecordFieldRemovalPath(final List<String> path) {
+        this.path = path;
+    }
+
+    public void add(final String fieldName) {
+        path.add(fieldName);
+    }
+
+    public int length() {
+        return path.size();
+    }
+
+    public String head() {
+        return path.get(path.size() - 1);
+    }
+
+    public RecordFieldRemovalPath tail() {
+        return new RecordFieldRemovalPath(path.subList(0, path.size() - 1));
+    }
+}
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
index cdc9a32fea..b2512afdd2 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java
@@ -87,4 +87,26 @@ public interface RecordSchema {
      */
     Optional<String> getSchemaNamespace();
 
+    /**
+     * @param fieldName the name of the field to be removed from the schema
+     */
+    void removeField(String fieldName);
+
+    /**
+     * @param path the sequence of field names to be removed from the schema
+     */
+    void removePath(RecordFieldRemovalPath path);
+
+    /**
+     * @return true if the schema contains itself as a nested field type, false if it does not
+     */
+    boolean isRecursive();
+
+    /**
+     * @param schemas the list of schemas to check whether the current schema is contained within
+     * @return true if the current schema is present within the list of schemas (object reference equality), false otherwise
+     */
+    default boolean sameAsAny(List<RecordSchema> schemas) {
+        return schemas.stream().anyMatch(schema -> schema == this); // reference equality check used on purpose
+    }
 }
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
index 26f491654a..5f51d05203 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java
@@ -18,8 +18,11 @@
 package org.apache.nifi.serialization.record.type;
 
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.util.List;
 import java.util.Objects;
 
 public class ArrayDataType extends DataType {
@@ -52,6 +55,20 @@ public class ArrayDataType extends DataType {
         return elementsNullable;
     }
 
+    @Override
+    public void removePath(final RecordFieldRemovalPath path) {
+        if (path.length() == 0) {
+            return;
+        }
+        getElementType().removePath(path.tail());
+    }
+
+    @Override
+    public boolean isRecursive(final List<RecordSchema> schemas) {
+        // allow for null elementType during schema inference
+        return getElementType() != null && getElementType().isRecursive(schemas);
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
index 9fcdf73f91..ce9072c498 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java
@@ -18,7 +18,9 @@
 package org.apache.nifi.serialization.record.type;
 
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
 
 import java.util.List;
 import java.util.Objects;
@@ -40,6 +42,23 @@ public class ChoiceDataType extends DataType {
         return RecordFieldType.CHOICE;
     }
 
+    @Override
+    public void removePath(final RecordFieldRemovalPath path) {
+        if (path.length() == 0) {
+            return;
+        }
+        getPossibleSubTypes().forEach(subType -> subType.removePath(path));
+    }
+
+    @Override
+    public boolean isRecursive(final List<RecordSchema> schemas) {
+        // allow for null possibleSubTypes during scheme inference
+        if (!schemas.isEmpty() && getPossibleSubTypes() != null) {
+            return getPossibleSubTypes().stream().anyMatch(possibleSubType -> possibleSubType.isRecursive(schemas));
+        }
+        return false;
+    }
+
     @Override
     public int hashCode() {
         return 31 + 41 * getFieldType().hashCode() + 41 * (possibleSubTypes == null ? 0 : possibleSubTypes.hashCode());
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
index 74c068325e..ced0b33aa8 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java
@@ -18,8 +18,11 @@
 package org.apache.nifi.serialization.record.type;
 
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordFieldType;
+import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.util.List;
 import java.util.Objects;
 
 public class MapDataType extends DataType {
@@ -52,6 +55,20 @@ public class MapDataType extends DataType {
         return RecordFieldType.MAP;
     }
 
+    @Override
+    public void removePath(final RecordFieldRemovalPath path) {
+        if (path.length() == 0) {
+            return;
+        }
+        getValueType().removePath(path.tail());
+    }
+
+    @Override
+    public boolean isRecursive(final List<RecordSchema> schemas) {
+        // allow for null valueType during scheme inference
+        return getValueType() != null && getValueType().isRecursive(schemas);
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) return true;
diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
index b49a6bf0a7..2e69e3e4fd 100644
--- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
+++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java
@@ -18,9 +18,12 @@
 package org.apache.nifi.serialization.record.type;
 
 import org.apache.nifi.serialization.record.DataType;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Objects;
 import java.util.function.Supplier;
 
@@ -54,6 +57,30 @@ public class RecordDataType extends DataType {
         return childSchema;
     }
 
+    @Override
+    public void removePath(final RecordFieldRemovalPath path) {
+        if (path.length() == 0) {
+            return;
+        }
+        getChildSchema().removePath(path);
+    }
+
+    @Override
+    public boolean isRecursive(final List<RecordSchema> schemas) {
+        // allow for childSchema to be null during schema inference
+        if (!schemas.isEmpty() && getChildSchema() != null) {
+            if (getChildSchema().sameAsAny(schemas)) {
+                return true;
+            } else {
+                final List<RecordSchema> schemasWithChildSchema = new ArrayList<>(schemas);
+                schemasWithChildSchema.add(getChildSchema());
+                return getChildSchema().getFields().stream()
+                        .anyMatch(childSchemaField -> childSchemaField.getDataType().isRecursive(schemasWithChildSchema));
+            }
+        }
+        return false;
+    }
+
     @Override
     public int hashCode() {
         return 31 + 41 * getFieldType().hashCode();
diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
index c4725ab04f..a858480013 100644
--- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
+++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java
@@ -17,11 +17,11 @@
 
 package org.apache.nifi.serialization;
 
+import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.SchemaIdentifier;
 import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
 
 import java.util.ArrayList;
 import java.util.Collections;
@@ -29,13 +29,14 @@ import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
 
-import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
 import static org.junit.jupiter.api.Assertions.assertThrows;
 
-public class TestSimpleRecordSchema {
+class TestSimpleRecordSchema {
 
     @Test
-    public void testPreventsTwoFieldsWithSameAlias() {
+    void testPreventsTwoFieldsWithSameAlias() {
         final List<RecordField> fields = new ArrayList<>();
         fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
         fields.add(new RecordField("goodbye", RecordFieldType.STRING.getDataType(), null, set("baz", "bar")));
@@ -44,7 +45,7 @@ public class TestSimpleRecordSchema {
     }
 
     @Test
-    public void testPreventsTwoFieldsWithSameName() {
+    void testPreventsTwoFieldsWithSameName() {
         final List<RecordField> fields = new ArrayList<>();
         fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
         fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType()));
@@ -53,7 +54,7 @@ public class TestSimpleRecordSchema {
     }
 
     @Test
-    public void testPreventsTwoFieldsWithConflictingNamesAliases() {
+    void testPreventsTwoFieldsWithConflictingNamesAliases() {
         final List<RecordField> fields = new ArrayList<>();
         fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar")));
         fields.add(new RecordField("bar", RecordFieldType.STRING.getDataType()));
@@ -62,7 +63,7 @@ public class TestSimpleRecordSchema {
     }
 
     @Test
-    public void testHashCodeAndEqualsWithSelfReferencingSchema() {
+    void testHashCodeAndEqualsWithSelfReferencingSchema() {
         final SimpleRecordSchema schema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
 
         final List<RecordField> personFields = new ArrayList<>();
@@ -70,39 +71,150 @@ public class TestSimpleRecordSchema {
         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() {
+        final String nameOfField1 = "field1";
+        final String nameOfField2 = "field2";
+        final DataType typeOfField1 = RecordFieldType.INT.getDataType();
+        final DataType typeOfField2 = RecordFieldType.STRING.getDataType();
+        final String schemaName = "schemaName";
+        final String namespace = "namespace";
+
+        final SimpleRecordSchema schema1 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName, namespace);
+        final SimpleRecordSchema schema2 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName, namespace);
+
+        assertEquals(schema1, schema2);
+    }
+
+    @Test
+    void testEqualsSimpleSchemaEvenIfSchemaNameAndNameSpaceAreDifferent() {
+        final String nameOfField1 = "field1";
+        final String nameOfField2 = "field2";
+        final DataType typeOfField1 = RecordFieldType.INT.getDataType();
+        final DataType typeOfField2 = RecordFieldType.STRING.getDataType();
+        final String schemaName1 = "schemaName1";
+        final String schemaName2 = "schemaName2";
+        final String namespace1 = "namespace1";
+        final String namespace2 = "namespace2";
+
+        final SimpleRecordSchema schema1 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName1, namespace1);
+        final SimpleRecordSchema schema2 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName2, namespace2);
+
+        assertEquals(schema1, schema2);
+    }
+
+    @Test
+    void testNotEqualsSimpleSchemaDifferentTypes() {
+        final String nameOfField1 = "field1";
+        final String nameOfField2 = "field2";
+        final DataType typeOfField1 = RecordFieldType.INT.getDataType();
+        final DataType typeOfField2 = RecordFieldType.STRING.getDataType();
+        final String schemaName = "schemaName";
+        final String namespace = "namespace";
+
+        final SimpleRecordSchema schema1 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField1, schemaName, namespace);
+        final SimpleRecordSchema schema2 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName, namespace);
+
+        assertNotEquals(schema1, schema2);
+    }
+
+    @Test
+    void testNotEqualsSimpleSchemaDifferentFieldNames() {
+        final String nameOfField1 = "fieldA";
+        final String nameOfField2 = "fieldB";
+        final String nameOfField3 = "fieldC";
+        final DataType typeOfField1 = RecordFieldType.INT.getDataType();
+        final DataType typeOfField2 = RecordFieldType.STRING.getDataType();
+        final String schemaName = "schemaName";
+        final String namespace = "namespace";
+
+        final SimpleRecordSchema schema1 = createSchemaWithTwoFields(nameOfField1, nameOfField2, typeOfField1, typeOfField2, schemaName, namespace);
+        final SimpleRecordSchema schema2 = createSchemaWithTwoFields(nameOfField1, nameOfField3, typeOfField1, typeOfField2, schemaName, namespace);
+
+        assertNotEquals(schema1, schema2);
     }
 
     @Test
-    public void testFieldsArentCheckedInEqualsIfNameAndNamespaceMatch() {
-        final RecordField testField = new RecordField("test", RecordFieldType.STRING.getDataType());
+    void testEqualsRecursiveSchema() {
+        final String field1 = "field1";
+        final String field2 = "field2";
+        final String schemaName = "schemaName";
+        final String namespace = "namespace";
 
-        final SimpleRecordSchema schema1 = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
-        schema1.setSchemaName("name");
-        schema1.setSchemaNamespace("namespace");
-        schema1.setFields(Collections.singletonList(testField));
+        final SimpleRecordSchema schema1 = createRecursiveSchema(field1, field2, schemaName, namespace);
+        final SimpleRecordSchema schema2 = createRecursiveSchema(field1, field2, schemaName, namespace);
 
-        SimpleRecordSchema schema2 = Mockito.spy(new SimpleRecordSchema(SchemaIdentifier.EMPTY));
-        schema2.setSchemaName("name");
-        schema2.setSchemaNamespace("namespace");
-        schema2.setFields(Collections.singletonList(testField));
+        assertEquals(schema1, schema2);
+        assertEquals(schema2, schema1);
+    }
+
+    @Test
+    void testNotEqualsRecursiveSchemaIfSchemaNameIsDifferent() {
+        final String field1 = "field1";
+        final String field2 = "field2";
+        final String schemaName1 = "schemaName1";
+        final String schemaName2 = "schemaName2";
+        final String namespace = "namespace";
+
+        final SimpleRecordSchema schema1 = createRecursiveSchema(field1, field2, schemaName1, namespace);
+        final SimpleRecordSchema schema2 = createRecursiveSchema(field1, field2, schemaName2, namespace);
+
+        assertThrows(StackOverflowError.class, () -> schema1.equals(schema2));
+        assertThrows(StackOverflowError.class, () -> schema2.equals(schema1));
+    }
+
+    @Test
+    void testNotEqualsRecursiveSchemaIfNamespaceIsDifferent() {
+        final String field1 = "fieldA";
+        final String field2 = "fieldB";
+        final String schemaName = "schemaName1";
+        final String namespace1 = "namespace1";
+        final String namespace2 = "namespace2";
+
+        final SimpleRecordSchema schema1 = createRecursiveSchema(field1, field2, schemaName, namespace1);
+        final SimpleRecordSchema schema2 = createRecursiveSchema(field1, field2, schemaName, namespace2);
+
+        assertThrows(StackOverflowError.class, () -> schema1.equals(schema2));
+        assertThrows(StackOverflowError.class, () -> schema2.equals(schema1));
+    }
 
-        assertTrue(schema1.equals(schema2));
-        Mockito.verify(schema2, Mockito.never()).getFields();
+    private SimpleRecordSchema createSchemaWithTwoFields(String nameOfField1, String nameOfField2,
+                                                         DataType typeOfField1, DataType typeOfField2,
+                                                         String schemaName, String schemaNamespace) {
+        final SimpleRecordSchema schema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField(nameOfField1, typeOfField1));
+        fields.add(new RecordField(nameOfField2, typeOfField2));
+        schema.setFields(fields);
+        schema.setSchemaName(schemaName);
+        schema.setSchemaNamespace(schemaNamespace);
+        return schema;
+    }
+
+    private SimpleRecordSchema createRecursiveSchema(String nameOfSimpleField, String nameOfRecursiveField,
+                                                     String schemaName, String schemaNamespace) {
+        final SimpleRecordSchema schema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField(nameOfSimpleField, RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField(nameOfRecursiveField, RecordFieldType.RECORD.getRecordDataType(schema)));
+        schema.setFields(fields);
+        schema.setSchemaName(schemaName);
+        schema.setSchemaNamespace(schemaNamespace);
+        return schema;
     }
 
     private Set<String> set(final String... values) {
         final Set<String> set = new HashSet<>();
-        for (final String value : values) {
-            set.add(value);
-        }
+        Collections.addAll(set, values);
         return set;
     }
 
diff --git a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/main/java/org/apache/nifi/accumulo/data/KeySchema.java b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/main/java/org/apache/nifi/accumulo/data/KeySchema.java
index 7ac74b833d..08800f6879 100644
--- a/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/main/java/org/apache/nifi/accumulo/data/KeySchema.java
+++ b/nifi-nar-bundles/nifi-accumulo-bundle/nifi-accumulo-processors/src/main/java/org/apache/nifi/accumulo/data/KeySchema.java
@@ -18,8 +18,10 @@
 package org.apache.nifi.accumulo.data;
 
 
+import org.apache.commons.lang3.NotImplementedException;
 import org.apache.nifi.serialization.record.DataType;
 import org.apache.nifi.serialization.record.RecordField;
+import org.apache.nifi.serialization.record.RecordFieldRemovalPath;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
 import org.apache.nifi.serialization.record.SchemaIdentifier;
@@ -112,4 +114,19 @@ public class KeySchema implements RecordSchema {
     public Optional<String> getSchemaNamespace() {
         return Optional.of("nifi-accumulo");
     }
+
+    @Override
+    public void removeField(String fieldName) {
+        throw new NotImplementedException("Field removal from Accumulo KeySchema is not implemented.");
+    }
+
+    @Override
+    public void removePath(RecordFieldRemovalPath path) {
+        throw new NotImplementedException("Path removal from Accumulo KeySchema is not implemented.");
+    }
+
+    @Override
+    public boolean isRecursive() {
+        throw new NotImplementedException("Determining if an Accumulo KeySchema is recursive is not implemented.");
+    }
 }
diff --git a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/TestJASN1RecordReaderWithComplexTypes.java b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/TestJASN1RecordReaderWithComplexTypes.java
index f90b63228a..c7f5957313 100644
--- a/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/TestJASN1RecordReaderWithComplexTypes.java
+++ b/nifi-nar-bundles/nifi-asn1-bundle/nifi-asn1-services/src/test/java/org/apache/nifi/jasn1/TestJASN1RecordReaderWithComplexTypes.java
@@ -35,13 +35,16 @@ import org.apache.nifi.serialization.record.Record;
 import org.apache.nifi.serialization.record.RecordField;
 import org.apache.nifi.serialization.record.RecordFieldType;
 import org.apache.nifi.serialization.record.RecordSchema;
+import org.apache.nifi.serialization.record.SchemaIdentifier;
 import org.apache.nifi.serialization.record.type.ArrayDataType;
 import org.apache.nifi.serialization.record.type.RecordDataType;
 import org.junit.jupiter.api.Test;
 
 import java.math.BigInteger;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.function.Function;
 
@@ -275,12 +278,15 @@ public class TestJASN1RecordReaderWithComplexTypes implements JASN1ReadRecordTes
          * The resolution of the recursive schema results in a cyclic reference graph which in turn leads to
          *  StackOverflowError when trying to compare to a similar resolved recursive schema.
          */
-        SimpleRecordSchema expectedSchema = new SimpleRecordSchema(Arrays.asList(
-                new RecordField("name", RecordFieldType.STRING.getDataType()),
-                new RecordField("children", RecordFieldType.ARRAY.getArrayDataType(
-                        RecordFieldType.RECORD.getRecordDataType(() -> null)
-                ))
+        final SimpleRecordSchema expectedSchema = new SimpleRecordSchema(SchemaIdentifier.EMPTY);
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("name", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("children", RecordFieldType.ARRAY.getArrayDataType(
+                RecordFieldType.RECORD.getRecordDataType(expectedSchema))
         ));
+        expectedSchema.setFields(fields);
+        expectedSchema.setSchemaName("Recursive");
+        expectedSchema.setSchemaNamespace("org.apache.nifi.jasn1.example");
 
         Map<String, Object> expectedValues = new HashMap<String, Object>() {{
             put("name", "name");
diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
index 7d10bacdd0..f5eec07965 100644
--- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java
@@ -54,29 +54,29 @@ import static org.apache.nifi.schema.access.SchemaAccessUtils.SCHEMA_TEXT_PROPER
 
 public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService {
 
-    static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Set 'schema.name' Attribute",
+    public static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Set 'schema.name' Attribute",
         "The FlowFile will be given an attribute named 'schema.name' and this attribute will indicate the name of the schema in the Schema Registry. Note that if"
             + "the schema for a record is not obtained from a Schema Registry, then no attribute will be added.");
-    static final AllowableValue AVRO_SCHEMA_ATTRIBUTE = new AllowableValue("full-schema-attribute", "Set 'avro.schema' Attribute",
+    public static final AllowableValue AVRO_SCHEMA_ATTRIBUTE = new AllowableValue("full-schema-attribute", "Set 'avro.schema' Attribute",
         "The FlowFile will be given an attribute named 'avro.schema' and this attribute will contain the Avro Schema that describes the records in the FlowFile. "
             + "The contents of the FlowFile need not be Avro, but the text of the schema will be used.");
-    static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference",
+    public static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference",
         "The content of the FlowFile will contain a reference to a schema in the Schema Registry service. The reference is encoded as a single byte indicating the 'protocol version', "
             + "followed by 8 bytes indicating the schema identifier, and finally 4 bytes indicating the schema version, as per the Hortonworks Schema Registry serializers and deserializers, "
             + "as found at https://github.com/hortonworks/registry. "
             + "This will be prepended to each FlowFile. Note that "
             + "if the schema for a record does not contain the necessary identifier and version, an Exception will be thrown when attempting to write the data.");
-    static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes",
+    public static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes",
         "The FlowFile will be given a set of 3 attributes to describe the schema: 'schema.identifier', 'schema.version', and 'schema.protocol.version'. Note that if "
             + "the schema for a record does not contain the necessary identifier and version, an Exception will be thrown when attempting to write the data.");
-    static final AllowableValue CONFLUENT_ENCODED_SCHEMA = new AllowableValue("confluent-encoded", "Confluent Schema Registry Reference",
+    public static final AllowableValue CONFLUENT_ENCODED_SCHEMA = new AllowableValue("confluent-encoded", "Confluent Schema Registry Reference",
         "The content of the FlowFile will contain a reference to a schema in the Schema Registry service. The reference is encoded as a single "
             + "'Magic Byte' followed by 4 bytes representing the identifier of the schema, as outlined at http://docs.confluent.io/current/schema-registry/docs/serializer-formatter.html. "
             + "This will be prepended to each FlowFile. Note that if the schema for a record does not contain the necessary identifier and version, "
             + "an Exception will be thrown when attempting to write the data. This is based on the encoding used by version 3.2.x of the Confluent Schema Registry.");
-    static final AllowableValue NO_SCHEMA = new AllowableValue("no-schema", "Do Not Write Schema", "Do not add any schema-related information to the FlowFile.");
+    public static final AllowableValue NO_SCHEMA = new AllowableValue("no-schema", "Do Not Write Schema", "Do not add any schema-related information to the FlowFile.");
 
-    static final PropertyDescriptor SCHEMA_CACHE = new Builder()
+    public static final PropertyDescriptor SCHEMA_CACHE = new Builder()
         .name("schema-cache")
         .displayName("Schema Cache")
         .description("Specifies a Schema Cache to add the Record Schema to so that Record Readers can quickly lookup the schema.")
@@ -147,7 +147,7 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic
         return INHERIT_RECORD_SCHEMA;
     }
 
-    protected PropertyDescriptor getSchemaWriteStrategyDescriptor() {
+    public PropertyDescriptor getSchemaWriteStrategyDescriptor() {
         return getPropertyDescriptor(SCHEMA_WRITE_STRATEGY.getName());
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index c213becf24..54ab75a288 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -504,74 +504,181 @@
                 <artifactId>apache-rat-plugin</artifactId>
                 <configuration>
                     <excludes combine.children="append">
-                        <exclude>src/test/resources/randombytes-1</exclude>
-                        <exclude>src/test/resources/localhost.cer</exclude>
-                        <exclude>src/test/resources/hello.txt</exclude>
                         <exclude>src/test/resources/CharacterSetConversionSamples/Converted.txt</exclude>
                         <exclude>src/test/resources/CharacterSetConversionSamples/Original.txt</exclude>
                         <exclude>src/test/resources/CompressedData/SampleFile.txt*</exclude>
+                        <exclude>src/test/resources/CompressedData/SampleFile1.txt*</exclude>
                         <exclude>src/test/resources/CompressedData/SampleFileConcat.txt*</exclude>
                         <exclude>src/test/resources/ExecuteCommand/1000bytes.txt</exclude>
                         <exclude>src/test/resources/ExecuteCommand/1mb.txt</exclude>
                         <exclude>src/test/resources/ExecuteCommand/test.txt</exclude>
+                        <exclude>src/test/resources/ExecuteCommand/TestDynamicEnvironment.jar</exclude>
+                        <exclude>src/test/resources/ExecuteCommand/TestIngestAndUpdate.jar</exclude>
+                        <exclude>src/test/resources/ExecuteCommand/TestLogStdErr.jar</exclude>
+                        <exclude>src/test/resources/ExecuteCommand/TestSuccess.jar</exclude>
+                        <exclude>src/test/resources/hello.txt</exclude>
+                        <exclude>src/test/resources/localhost.cer</exclude>
+                        <exclude>src/test/resources/randombytes-1</exclude>
                         <exclude>src/test/resources/ScanAttribute/dictionary-with-empty-new-lines</exclude>
                         <exclude>src/test/resources/ScanAttribute/dictionary-with-extra-info</exclude>
                         <exclude>src/test/resources/ScanAttribute/dictionary1</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/text.txt</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/text.txt.asc</exclude>
-                        <exclude>src/test/resources/TestGenerateRecord/nested_no_nullable.avsc</exclude>
-                        <exclude>src/test/resources/TestGenerateRecord/nested_nullable.avsc</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.csv</exclude>
-                        <exclude>src/test/resources/TestJson/json-sample.json</exclude>
-                        <exclude>src/test/resources/TestJson/control-characters.json</exclude>
-                        <exclude>src/test/resources/TestMergeContent/demarcate</exclude>
-                        <exclude>src/test/resources/TestMergeContent/foot</exclude>
-                        <exclude>src/test/resources/TestMergeContent/head</exclude>
-                        <exclude>src/test/resources/TestMergeContent/user.avsc</exclude>
-                        <exclude>src/test/resources/TestMergeContent/place.avsc</exclude>
                         <exclude>src/test/resources/TestConversions/data.int_float_string.json</exclude>
                         <exclude>src/test/resources/TestConversions/data.int_float_string.with_header.csv</exclude>
-                        <exclude>src/test/resources/TestConversions/data.int_float_string.without_header.csv</exclude>
-                        <exclude>src/test/resources/TestConversions/data.int_float_string.xml</exclude>
                         <exclude>src/test/resources/TestConversions/data.int_float_string.with_schema.avro</exclude>
                         <exclude>src/test/resources/TestConversions/data.int_float_string.with_schema.json.to.avro</exclude>
+                        <exclude>src/test/resources/TestConversions/data.int_float_string.without_header.csv</exclude>
                         <exclude>src/test/resources/TestConversions/data.int_float_string.without_schema.avro</exclude>
+                        <exclude>src/test/resources/TestConversions/data.int_float_string.xml</exclude>
                         <exclude>src/test/resources/TestConversions/explicit.schema.json</exclude>
-                        <exclude>src/test/resources/TestConvertJSONToSQL/person-1.json</exclude>
-                        <exclude>src/test/resources/TestConvertJSONToSQL/persons.json</exclude>
                         <exclude>src/test/resources/TestConvertJSONToSQL/malformed-person-extra-comma.json</exclude>
+                        <exclude>src/test/resources/TestConvertJSONToSQL/person-1.json</exclude>
+                        <exclude>src/test/resources/TestConvertJSONToSQL/person-with-bool.json</exclude>
                         <exclude>src/test/resources/TestConvertJSONToSQL/person-with-extra-field.json</exclude>
-                        <exclude>src/test/resources/TestConvertJSONToSQL/person-without-code.json</exclude>
                         <exclude>src/test/resources/TestConvertJSONToSQL/person-with-null-code.json</exclude>
+                        <exclude>src/test/resources/TestConvertJSONToSQL/person-without-code.json</exclude>
                         <exclude>src/test/resources/TestConvertJSONToSQL/person-without-id.json</exclude>
-                        <exclude>src/test/resources/TestConvertJSONToSQL/person-with-bool.json</exclude>
+                        <exclude>src/test/resources/TestConvertJSONToSQL/persons.json</exclude>
+                        <exclude>src/test/resources/TestConvertRecord/input/person_bad_enum.json</exclude>
+                        <exclude>src/test/resources/TestConvertRecord/input/person_long_id.json</exclude>
+                        <exclude>src/test/resources/TestConvertRecord/input/person.json</exclude>
+                        <exclude>src/test/resources/TestConvertRecord/schema/person_with_union_enum_string.avsc</exclude>
+                        <exclude>src/test/resources/TestConvertRecord/schema/person.avsc</exclude>
                         <exclude>src/test/resources/TestCountText/jabberwocky.txt</exclude>
-                        <exclude>src/test/resources/TestModifyBytes/noFooter.txt</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/plain.txt</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/salted_128_raw.enc</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/salted_raw.enc</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/text.txt.asc</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/text.txt</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/unsalted_128_raw.enc</exclude>
+                        <exclude>src/test/resources/TestEncryptContent/unsalted_raw.enc</exclude>
+                        <exclude>src/test/resources/TestExtractGrok/apache.log</exclude>
+                        <exclude>src/test/resources/TestExtractGrok/patterns</exclude>
+                        <exclude>src/test/resources/TestExtractGrok/simple_text.log</exclude>
+                        <exclude>src/test/resources/TestForkRecord/input/complex-input-json.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/output/extract-transactions.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/output/split-address.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/output/split-transactions.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/schema/extract-schema.avsc</exclude>
+                        <exclude>src/test/resources/TestForkRecord/schema/schema.avsc</exclude>
+                        <exclude>src/test/resources/TestForkRecord/single-element-nested-array-strings.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/single-element-nested-array.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/single-element-nested-nested-array.json</exclude>
+                        <exclude>src/test/resources/TestForkRecord/two-elements-nested-nested-array-null.json</exclude>
+                        <exclude>src/test/resources/TestGenerateRecord/nested_no_nullable.avsc</exclude>
+                        <exclude>src/test/resources/TestGenerateRecord/nested_nullable.avsc</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/.customConfig.xml</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.csv</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.jar</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.tar.gz</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.tar</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt.bz2</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt.gz</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/1.zip</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/2.custom</exclude>
+                        <exclude>src/test/resources/TestIdentifyMimeType/flowfilev1.tar</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/insert-enrichment-first-value-null.json</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/insert-enrichment.json</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/insert-original.json</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-enrichment.csv</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-expected.csv</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-original.csv</exclude>
+                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-rename-expected.csv</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/cardrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/cardrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/chainrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/chainrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/customChainrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/defaultrELOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/defaultrELSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/defaultrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/defaultrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/input.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefaultOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefaultSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefineOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefineSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierOverwriteOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/modifierOverwriteSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/removrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/removrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/shiftrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/shiftrSpec.json</exclude>
+                        <exclude>src/test/resources/TestJoltTransformJson/sortrOutput.json</exclude>
+                        <exclude>src/test/resources/TestJson/control-characters.json</exclude>
+                        <exclude>src/test/resources/TestJson/json-sample.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-input-unmatched.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-input.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-output-unmatched.json</exclude>
+                        <exclude>src/test/resources/TestLookupRecord/lookup-array-output.json</exclude>
+                        <exclude>src/test/resources/TestMergeContent/demarcate</exclude>
+                        <exclude>src/test/resources/TestMergeContent/foot</exclude>
+                        <exclude>src/test/resources/TestMergeContent/head</exclude>
+                        <exclude>src/test/resources/TestMergeContent/place.avsc</exclude>
+                        <exclude>src/test/resources/TestMergeContent/user.avsc</exclude>
                         <exclude>src/test/resources/TestModifyBytes/noFooter_noHeader.txt</exclude>
+                        <exclude>src/test/resources/TestModifyBytes/noFooter.txt</exclude>
                         <exclude>src/test/resources/TestModifyBytes/noHeader.txt</exclude>
                         <exclude>src/test/resources/TestModifyBytes/testFile.txt</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input_schema/address-array.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input_schema/address-choice.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input_schema/bookshelf-map.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input_schema/complex-person.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input_schema/name-choice.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input/address-array.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input/address-choice.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input/bookshelf-map.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input/complex-person.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/input/name-choice.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/address-array-field-removed.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/address-array-zip-removed.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/address-array.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/address-choice-letter-removed-from-building.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map-author-field-removed.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/complex-person-multiple-fields-removed.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-dateOfBirth.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress-building-letter.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/complex-person.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output_schema/name-choice-firstName-removed.avsc</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-array-empty.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-array-one-element-removed.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-array-removed.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-all-elements.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-one-element.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/address-choice-letter-removed-from-building.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/bookshelf-map-all-items-removed.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-all-elements.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-one-element.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/bookshelf-map-one-item-removed.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/complex-person-multiple-fields-removed.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/complex-person-no-dateOfBirth.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress-building-letter.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/complex-person.json</exclude>
+                        <exclude>src/test/resources/TestRemoveRecordField/output/name-choice-firstName-removed.json</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/[DODO].txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/$1$1.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/Blu$2e_clu$2e.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/Good.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/Spider.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/[DODO].txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/BRue_cRue_RiRey.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/cu_Po.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/cu[$1]_Po[$1].txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/D$d_h$d.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/food.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/testFile.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/AppendLineByLineTest.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/Good.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptFirstLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptLastLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceFirstLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceLastLine.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/PrependLineByLineTest.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceLastLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceFirstLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceExceptLastLine.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceExceptFirstLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceLastLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceFirstLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptLastLine.txt</exclude>
-                        <exclude>src/test/resources/TestReplaceTextLineByLine/LiteralReplaceExceptFirstLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceExceptLastLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceFirstLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/ReplaceLastLine.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/Spider.txt</exclude>
+                        <exclude>src/test/resources/TestReplaceTextLineByLine/testFile.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-backreference-mapping.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-blank-mapping.txt</exclude>
                         <exclude>src/test/resources/TestReplaceTextWithMapping/color-fruit-escaped-dollar-mapping.txt</exclude>
@@ -591,137 +698,67 @@
                         <exclude>src/test/resources/TestSplitText/4.txt</exclude>
                         <exclude>src/test/resources/TestSplitText/5.txt</exclude>
                         <exclude>src/test/resources/TestSplitText/6.txt</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/input.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/chainrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/customChainrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/chainrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/cardrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/cardrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/shiftrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/sortrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/shiftrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/removrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/removrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrELSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/defaultrELOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefaultSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefaultOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefineSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierDefineOutput.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierOverwriteSpec.json</exclude>
-                        <exclude>src/test/resources/TestJoltTransformJson/modifierOverwriteOutput.json</exclude>
                         <exclude>src/test/resources/TestSplitText/original.txt</exclude>
+                        <exclude>src/test/resources/TestTransformXml/employee.html</exclude>
                         <exclude>src/test/resources/TestTransformXml/math.html</exclude>
                         <exclude>src/test/resources/TestTransformXml/tokens.csv</exclude>
                         <exclude>src/test/resources/TestTransformXml/tokens.xml</exclude>
-                        <exclude>src/test/resources/TestTransformXml/employee.html</exclude>
-                        <exclude>src/test/resources/TestUnpackContent/folder/cal.txt</exclude>
-                        <exclude>src/test/resources/TestUnpackContent/folder/date.txt</exclude>
                         <exclude>src/test/resources/TestUnpackContent/data.flowfilev2</exclude>
                         <exclude>src/test/resources/TestUnpackContent/data.flowfilev3</exclude>
-                        <exclude>src/test/resources/TestXml/xml-bundle-1</exclude>
-                        <exclude>src/test/resources/TestXml/namespaceSplit1.xml</exclude>
-                        <exclude>src/test/resources/TestXml/namespaceSplit2.xml</exclude>
-                        <exclude>src/test/resources/ExecuteCommand/TestIngestAndUpdate.jar</exclude>
-                        <exclude>src/test/resources/ExecuteCommand/TestSuccess.jar</exclude>
-                        <exclude>src/test/resources/ExecuteCommand/TestDynamicEnvironment.jar</exclude>
-                        <exclude>src/test/resources/ExecuteCommand/TestLogStdErr.jar</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.jar</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.tar</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.tar.gz</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt.bz2</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.txt.gz</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/1.zip</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/flowfilev1.tar</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/2.custom</exclude>
-                        <exclude>src/test/resources/TestIdentifyMimeType/.customConfig.xml</exclude>
                         <exclude>src/test/resources/TestUnpackContent/data.tar</exclude>
                         <exclude>src/test/resources/TestUnpackContent/data.zip</exclude>
+                        <exclude>src/test/resources/TestUnpackContent/folder/cal.txt</exclude>
+                        <exclude>src/test/resources/TestUnpackContent/folder/date.txt</exclude>
                         <exclude>src/test/resources/TestUnpackContent/invalid_data.zip</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/plain.txt</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/salted_raw.enc</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/salted_128_raw.enc</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/unsalted_raw.enc</exclude>
-                        <exclude>src/test/resources/TestEncryptContent/unsalted_128_raw.enc</exclude>
-                        <exclude>src/test/resources/TestExtractGrok/apache.log</exclude>
-                        <exclude>src/test/resources/TestExtractGrok/simple_text.log</exclude>
-                        <exclude>src/test/resources/TestExtractGrok/patterns</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/input/addresses.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/input/embedded-string.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/input/person.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/input/multi-arrays.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/input/person-address.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/input/person-stringified-name.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/input/person-with-null-array.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/input/multi-arrays.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/input/person.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/output/embedded-string.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-null-array.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-firstname.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-firstname-lastname.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-capital-lastname.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/name-fields-only.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/full-addresses.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/output/name-and-mother-same.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/name-fields-only.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-capital-lastname.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-firstname-lastname.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-firstname.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/output/person-with-name.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/output/person-with-new-city.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/person-with-null-array.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-0and1.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-0and2.json</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-streets.json</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/schema/embedded-record.avsc</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/schema/name-fields-only.avsc</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/schema/person-with-address.avsc</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-and-mother.avsc</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-record.avsc</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-string.avsc</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-string-fields.avsc</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/schema/name-fields-only.avsc</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-and-mother.avsc</exclude>
+                        <exclude>src/test/resources/TestUpdateRecord/schema/person-with-name-string.avsc</exclude>
                         <exclude>src/test/resources/TestUpdateRecord/schema/person-with-stringified-name.avsc</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/schema/multi-arrays.avsc</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-0and1.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-0and2.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/updateArrays/multi-arrays-streets.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/input/addresses.json</exclude>
-                        <exclude>src/test/resources/TestUpdateRecord/output/full-addresses.json</exclude>
-                        <exclude>src/test/resources/xxe_template.xml</exclude>
-                        <exclude>src/test/resources/xxe_from_report.xml</exclude>
-                        <exclude>src/test/resources/TestForkRecord/single-element-nested-array-strings.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/single-element-nested-array.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/single-element-nested-nested-array.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/two-elements-nested-nested-array-null.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/input/complex-input-json.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/output/extract-transactions.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/output/split-address.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/output/split-transactions.json</exclude>
-                        <exclude>src/test/resources/TestForkRecord/schema/extract-schema.avsc</exclude>
-                        <exclude>src/test/resources/TestForkRecord/schema/schema.avsc</exclude>
-                        <exclude>src/test/resources/TestConvertRecord/schema/person.avsc</exclude>
-                        <exclude>src/test/resources/TestConvertRecord/schema/person_with_union_enum_string.avsc</exclude>
-                        <exclude>src/test/resources/TestConvertRecord/input/person.json</exclude>
-                        <exclude>src/test/resources/TestConvertRecord/input/person_bad_enum.json</exclude>
-                        <exclude>src/test/resources/TestConvertRecord/input/person_long_id.json</exclude>
-                        <exclude>src/test/resources/TestValidateRecord/missing-array.json</exclude>
-                        <exclude>src/test/resources/TestValidateRecord/missing-array.avsc</exclude>
+                        <exclude>src/test/resources/TestValidateJson/schema-simple-example-missing-required.json</exclude>
+                        <exclude>src/test/resources/TestValidateJson/schema-simple-example-unmatched-pattern.json</exclude>
+                        <exclude>src/test/resources/TestValidateJson/schema-simple-example.json</exclude>
+                        <exclude>src/test/resources/TestValidateJson/simple-example-with-comments.json</exclude>
+                        <exclude>src/test/resources/TestValidateJson/simple-example.json</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/int-maps-data.json</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/int-maps-schema.avsc</exclude>
                         <exclude>src/test/resources/TestValidateRecord/missing-array-with-default.avsc</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/missing-array.avsc</exclude>
+                        <exclude>src/test/resources/TestValidateRecord/missing-array.json</exclude>
                         <exclude>src/test/resources/TestValidateRecord/nested-map-input.json</exclude>
                         <exclude>src/test/resources/TestValidateRecord/nested-map-schema.avsc</exclude>
                         <exclude>src/test/resources/TestValidateRecord/timestamp.avsc</exclude>
                         <exclude>src/test/resources/TestValidateRecord/timestamp.json</exclude>
-                        <exclude>src/test/resources/TestLookupRecord/lookup-array-input.json</exclude>
-                        <exclude>src/test/resources/TestLookupRecord/lookup-array-input-unmatched.json</exclude>
-                        <exclude>src/test/resources/TestLookupRecord/lookup-array-output.json</exclude>
-                        <exclude>src/test/resources/TestLookupRecord/lookup-array-output-unmatched.json</exclude>
-                        <exclude>src/test/resources/TestValidateRecord/int-maps-schema.avsc</exclude>
-                        <exclude>src/test/resources/TestValidateRecord/int-maps-data.json</exclude>
-                        <exclude>src/test/resources/TestValidateRecord/array-and-map-with-null-element.avro</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/insert-enrichment.json</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/insert-enrichment-first-value-null.json</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/insert-original.json</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-enrichment.csv</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-expected.csv</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-original.csv</exclude>
-                        <exclude>src/test/resources/TestJoinEnrichment/left-outer-join-rename-expected.csv</exclude>
-                        <exclude>src/test/resources/TestValidateJson/schema-simple-example-missing-required.json</exclude>
-                        <exclude>src/test/resources/TestValidateJson/schema-simple-example-unmatched-pattern.json</exclude>
-                        <exclude>src/test/resources/TestValidateJson/schema-simple-example.json</exclude>
-                        <exclude>src/test/resources/TestValidateJson/simple-example.json</exclude>
-                        <exclude>src/test/resources/TestValidateJson/simple-example-with-comments.json</exclude>
+                        <exclude>src/test/resources/TestXml/namespaceSplit1.xml</exclude>
+                        <exclude>src/test/resources/TestXml/namespaceSplit2.xml</exclude>
+                        <exclude>src/test/resources/TestXml/xml-bundle-1</exclude>
+                        <exclude>src/test/resources/xxe_from_report.xml</exclude>
+                        <exclude>src/test/resources/xxe_template.xml</exclude>
                     </excludes>
                 </configuration>
             </plugin>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java
index 8d4466ccdd..d7ff87ac04 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ForkRecord.java
@@ -267,9 +267,12 @@ public class ForkRecord extends AbstractProcessor {
                                         RecordFieldType fieldType = fieldValue.getField().getDataType().getFieldType();
 
                                         // we want to have an array here, nothing else allowed
-                                        if(fieldType != RecordFieldType.ARRAY) {
-                                            getLogger().debug("The record path " + recordPath.getPath() + " is matching a field "
-                                                    + "of type " + fieldType + " when the type ARRAY is expected.");
+                                        if (fieldType != RecordFieldType.ARRAY) {
+                                            getLogger().debug("The record path {} is matching a field of type {} when the type ARRAY is expected.", recordPath.getPath(), fieldType);
+                                            continue;
+                                        }
+                                        if (fieldValue.getValue() == null) {
+                                            getLogger().debug("The record path {} is matching a field the value of which is null.", recordPath.getPath());
                                             continue;
                                         }
 
@@ -288,9 +291,9 @@ public class ForkRecord extends AbstractProcessor {
                                             final DataType elementType = arrayDataType.getElementType();
 
                                             // we want to have records in the array
-                                            if(elementType.getFieldType() != RecordFieldType.RECORD) {
-                                                getLogger().debug("The record path " + recordPath.getPath() + " is matching an array field with "
-                                                        + "values of type " + elementType.getFieldType() + " when the type RECORD is expected.");
+                                            if (elementType.getFieldType() != RecordFieldType.RECORD) {
+                                                getLogger().debug("The record path {} is matching an array field with values of type {} " +
+                                                        "when the type RECORD is expected.", recordPath.getPath(), elementType.getFieldType());
                                                 continue;
                                             }
 
@@ -325,7 +328,7 @@ public class ForkRecord extends AbstractProcessor {
                             try {
                                 recordSetWriter.close();
                             } catch (final IOException ioe) {
-                                getLogger().warn("Failed to close Writer for {}", new Object[] {outFlowFile});
+                                getLogger().warn("Failed to close Writer for {}", outFlowFile);
                             }
 
                             final Map<String, String> attributes = new HashMap<>();
@@ -367,7 +370,7 @@ public class ForkRecord extends AbstractProcessor {
             });
 
         } catch (Exception e) {
-            getLogger().error("Failed to fork {}", new Object[] {flowFile, e});
+            getLogger().error("Failed to fork {}", flowFile, e);
             session.remove(outFlowFile);
             session.transfer(original, REL_FAILURE);
             return;
@@ -375,7 +378,7 @@ public class ForkRecord extends AbstractProcessor {
 
         session.adjustCounter("Records Processed", readCount.get(), false);
         session.adjustCounter("Records Generated", writeCount.get(), false);
-        getLogger().debug("Successfully forked {} records into {} records in {}", new Object[] {readCount.get(), writeCount.get(), flowFile});
+        getLogger().debug("Successfully forked {} records into {} records in {}", readCount.get(), writeCount.get(), flowFile);
         session.transfer(original, REL_ORIGINAL);
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RemoveRecordField.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/RemoveRecordField.java
new file mode 100644
index 0000000000..7a5cccec71
--- /dev/null
+++ b/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.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;
+
+
+@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;
+
+    private static final String ROOT_PATH = "/";
+
+    @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 (ROOT_PATH.equals(path)) {
+                            validationResults.add(new ValidationResult.Builder()
+                                    .subject(property.getDisplayName()).valid(false)
+                                    .explanation("the root RecordPath cannot be removed").build()
+                            );
+                        }
+                    });
+            return validationResults;
+        }
+
+        return Collections.singleton(new ValidationResult.Builder()
+                .subject("User-defined Properties")
+                .valid(false)
+                .explanation("at least one RecordPath must be specified")
+                .build());
+    }
+
+    @OnScheduled
+    public void collectRecordPaths(final ProcessContext context) {
+        recordPathCache = new RecordPathCache(context.getProperties().size() * 2);
+
+        recordPathsToRemove = null;
+    }
+
+    @Override
+    protected Record process(final Record record, final FlowFile flowFile, final ProcessContext context, final long count) {
+        if (recordPathsToRemove == null) {
+            recordPathsToRemove = new ArrayList<>(context.getProperties().size());
+            context.getProperties().keySet().forEach(property -> {
+                if (property.isDynamic()) {
+                    // validate RecordPath from Expression Language (if applicable)
+                    final String recordPath = context.getProperty(property).evaluateAttributeExpressions(flowFile).getValue();
+                    if (ROOT_PATH.equals(recordPath)) {
+                        throw new ProcessException(String.format("The root Record Path %s cannot be removed for %s", ROOT_PATH, property.getDisplayName()));
+                    }
+                    recordPathsToRemove.add(new RecordFieldRemover.RecordPathRemovalProperties(recordPath));
+                }
+            });
+        }
+
+        final RecordFieldRemover recordFieldRemover = new RecordFieldRemover(record, recordPathCache);
+        recordPathsToRemove.forEach(recordFieldRemover::remove);
+        return recordFieldRemover.getRecord();
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
index c5607b3060..eb01b54c9f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/META-INF/services/org.apache.nifi.processor.Processor
@@ -98,6 +98,7 @@ org.apache.nifi.processors.standard.PutUDP
 org.apache.nifi.processors.standard.QueryDatabaseTable
 org.apache.nifi.processors.standard.QueryDatabaseTableRecord
 org.apache.nifi.processors.standard.QueryRecord
+org.apache.nifi.processors.standard.RemoveRecordField
 org.apache.nifi.processors.standard.ReplaceText
 org.apache.nifi.processors.standard.ReplaceTextWithMapping
 org.apache.nifi.processors.standard.RetryFlowFile
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RemoveRecordField/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RemoveRecordField/additionalDetails.html
new file mode 100644
index 0000000000..4d062c9fae
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.RemoveRecordField/additionalDetails.html
@@ -0,0 +1,633 @@
+<!DOCTYPE html>
+<html lang="en">
+<!--
+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.
+-->
+<head>
+    <meta charset="utf-8"/>
+    <title>RemoveRecordField</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+    <style>		table td:first-child {text-align: center;}	</style>
+</head>
+
+<body>
+<h2>RemoveRecordField processor usage with examples</h2>
+
+<p>
+    The RemoveRecordField processor is capable of removing fields from a
+    NiFi record. The fields that should be removed from the record are
+    identified by a RecordPath expression. To learn about RecordPath,
+    please read the
+    <a class="document-link record-path-guide" href="../../../../../html/record-path-guide.html" target="component-usage">RecordPath Guide</a>.
+</p>
+<p>
+    RemoveRecordField will update all Records within the FlowFile based
+    upon the RecordPath(s) configured for removal. The Schema associated
+    with the Record Reader configured to read the FlowFile content will
+    be updated based upon the same RecordPath(s) and considering the values
+    remaining within the Record's Fields after removal.
+
+    This updated schema can be used for output if the Record Writer has a
+    Schema Access Strategy of Inherit Record Schema, otherwise the schema
+    updates will be lost and the Records output using the Schema configured
+    upon the Writer.
+</p>
+<p>
+    Below are some examples that are intended to explain how to use the
+    processor. In these examples the input data, the record schema, the
+    output data and the output schema are all in JSON format for easy
+    understanding.
+
+    We assume that the processor is configured to use a
+    JsonTreeReader and JsonRecordSetWriter controller service, but of
+    course the processor works with other Reader and Writer controller
+    services as well.
+
+    In the examples it is also assumed that the record
+    schema is provided explicitly as a FlowFile attribute (avro.schema
+    attribute), and the Reader uses this schema to work with the
+    FlowFile.
+
+    The Writer's Schema Access Strategy is "Inherit Record Schema"
+    so that all modifications made to the schema by the processor
+    are considered by the Writer. Schema Write Strategy of the Writer is
+    set to "Set 'avro.schema' Attribute" so that the output FlowFile
+    contains the schema as an attribute value.
+</p>
+<h3><u>Example 1:</u></h3>
+<p>
+    <b>Removing a field from a simple record</b>
+</p>
+
+<p>Input data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "dateOfBirth": "1980-01-01"
+}</pre>
+
+<p>Input schema:</p>
+<pre>{
+    "type": "record",
+    "name": "PersonRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "dateOfBirth",
+            "type": "string"
+        }
+    ]
+}</pre>
+<p>Field to remove:</p>
+<pre>/dateOfBirth</pre>
+
+<p>In this case the <i>dateOfBirth</i> field is removed from the record
+    as well as the schema.</p>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe"
+}</pre>
+
+<p>Output schema:</p>
+<pre>{
+    "type": "record",
+    "name": "PersonRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        }
+    ]
+}</pre>
+
+<p>Note, that removing a field from a record differs from setting a
+    field's value to <i>null</i>. With RemoveRecordField a field is
+    completely removed from the record and its schema regardless of the
+    field being nullable or not.</p>
+
+
+<h3><u>Example 2:</u></h3>
+<p>
+    <b>Removing fields from a complex record</b>
+</p>
+<p>Let's suppose we have an input record that contains a
+    <i>homeAddress</i> and a <i>mailingAddress</i> field both of which
+    contain a <i>zip</i> field and we want to remove the <i>zip</i>
+    field from both of them.</p>
+<p>Input data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "homeAddress": {
+        "zip": 1111,
+        "street": "Main",
+        "number": 24
+    },
+    "mailingAddress": {
+        "zip": 1121,
+        "street": "Airport",
+        "number": 12
+    }
+}</pre>
+
+<p>Input schema:</p>
+<pre>{
+    "name": "PersonRecord",
+    "type": "record",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "homeAddress",
+            "type": {
+                "name": "address",
+                "type": "record",
+                "fields": [
+                    {
+                        "name": "zip",
+                        "type": "int"
+                    },
+                    {
+                        "name": "street",
+                        "type": "string"
+                    },
+                    {
+                        "name": "number",
+                        "type": "int"
+                    }
+                ]
+            }
+        },
+        {
+            "name": "mailingAddress",
+            "type": "address"
+        }
+    ]
+}</pre>
+<p>The <i>zip</i> field from both addresses can be removed by
+    specifying "Field to remove 1" property on the processor as
+    "/homeAddress/zip" and adding a dynamic property with the value
+    "/mailingAddress/zip". Or we can use a wildcard expression in the
+    RecordPath in "Field To Remove 1" (no need to specify a dynamic
+    property).</p>
+<p>Field to remove:</p>
+<pre>/*/zip</pre>
+
+<p>The <i>zip</i> field is removed from both addresses.</p>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "homeAddress": {
+        "street": "Main",
+        "number": 24
+    },
+    "mailingAddress": {
+        "street": "Airport",
+        "number": 12
+    }
+}</pre>
+<p>The <i>zip</i> field is removed from the schema of both the
+    <i>homeAddress</i> field and the <i>mailingAddress</i> field.
+    However, if only "/homeAddress/zip" was specified to be removed,
+    the schema of <i>mailingAddress</i> would be intact regardless of
+    the fact that originally these two addresses shared the same
+    schema.</p>
+
+<h3><u>Example 3:</u></h3>
+<p>
+    <b>Arrays</b>
+</p>
+<p>Let's suppose we have an input record that contains an array of
+    addresses.</p>
+<p>Input data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "addresses": [
+        {
+            "zip": 1111,
+            "street": "Main",
+            "number": 24
+        },
+        {
+            "zip": 1121,
+            "street": "Airport",
+            "number": 12
+        }
+    ]
+}</pre>
+
+<p>Input schema:</p>
+<pre>{
+    "name": "PersonRecord",
+    "type": "record",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "addresses",
+            "type": {
+                "type": "array",
+                "items": {
+                    "name": "address",
+                    "type": "record",
+                    "fields": [
+                        {
+                            "name": "zip",
+                            "type": "int"
+                        },
+                        {
+                            "name": "street",
+                            "type": "string"
+                        },
+                        {
+                            "name": "number",
+                            "type": "int"
+                        }
+                    ]
+                }
+            }
+        }
+    ]
+}</pre>
+
+<ul><li>Case 1: removing one element from the array</li></ul>
+<p>Field to remove:</p>
+<pre>/addresses[0]</pre>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "addresses": [
+        {
+            "zip": 1121,
+            "street": "Airport",
+            "number": 12
+        }
+    ]
+}</pre>
+<p>Output schema:</p>
+<pre>{
+    "type": "record",
+    "name": "nifiRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "addresses",
+            "type": {
+                "type": "array",
+                "items": {
+                    "type": "record",
+                    "name": "addressesType",
+                    "fields": [
+                        {
+                            "name": "zip",
+                            "type": "int"
+                        },
+                        {
+                            "name": "street",
+                            "type": "string"
+                        },
+                        {
+                            "name": "number",
+                            "type": "int"
+                        }
+                    ]
+                }
+            }
+        }
+    ]
+}</pre>
+<p>The first element of the array is removed. The schema of the output
+    data is structurally the same as the input schema. Note that the name
+    "PersonRecord" of the input schema changed to "nifiRecord" and
+    the name "address" changed to "addressesType". This is normal, NiFi
+    generates these names for the output schema. These name changes occur
+    regardless of the schema actually being modified or not.</p>
+
+<ul><li>Case 2: removing all elements from the array</li></ul>
+<p>Field to remove:</p>
+<pre>/addresses[*]</pre>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "addresses": []
+}</pre>
+<p>All elements of the array are removed, the result is an empty array.
+    The output schema is the same as in Case 1, no structural changes
+    made to the schema.</p>
+
+<ul><li>Case 3: removing a field from certain elements of the array</li></ul>
+<p>Field to remove:</p>
+<pre>/addresses[0]/zip</pre>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "addresses": [
+        {
+            "zip": null,
+            "street": "Main",
+            "number": 24
+        },
+        {
+            "zip": 1121,
+            "street": "Airport",
+            "number": 12
+        }
+    ]
+}</pre>
+
+<p>The output schema is the same as in Case 1, no structural changes. The
+    <i>zip</i> field of the array's first element is set to <i>null</i>
+    since the value had to be deleted but the schema could not be
+    modified since deletion is not applied to all elements in the array.
+    In a case like this, the value of the field is set to <i>null</i>
+    regardless of the field being nullable or not.</p>
+
+<ul><li>Case 4: removing a field from all elements of an array</li></ul>
+<p>Field to remove:</p>
+<pre>/addresses[*]/zip</pre>
+
+<p>Output data:</p>
+<pre>{
+    "id": 1,
+    "name": "John Doe",
+    "addresses": [
+        {
+            "street": "Main",
+            "number": 24
+        },
+        {
+            "street": "Airport",
+            "number": 12
+        }
+    ]
+}</pre>
+<p>Output schema:</p>
+<pre>{
+    "type": "record",
+    "name": "nifiRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "addresses",
+            "type": {
+                "type": "array",
+                "items": {
+                    "type": "record",
+                    "name": "addressesType",
+                    "fields": [
+                        {
+                            "name": "street",
+                            "type": "string"
+                        },
+                        {
+                            "name": "number",
+                            "type": "int"
+                        }
+                    ]
+                }
+            }
+        }
+    ]
+}</pre>
+<p>The <i>zip</i> field is removed from all elements of the array, and the
+    schema is modified, the <i>zip</i> field is removed from the array's
+    element type.</p>
+
+<p>The examples shown in Case 1, Case 2, Case 3 and Case 4 apply to both
+    kinds of collections: arrays and maps. The schema of an array or a
+    map is only modified if the field removal applies to all elements of
+    the collection. Selecting all elements of an array can be performed
+    with the [*] as well as the [0..-1] operator.</p>
+
+<p><b>Important note: </b> if there are e.g. 3 elements in the addresses
+    array, and "/addresses[*]/zip" is removed, then the <i>zip</i> field
+    is removed from the schema because it applies explicitly for all
+    elements regardless of the actual number of elements in the array.
+    However, if the path says "/addresses[0,1,2]/zip" then the schema is
+    NOT modified (even though [0,1,2] means all the elements in this
+    particular array), because it selects the first, second and third
+    elements individually and does not express the intention to apply
+    removal to all elements of the array regardless of the number of
+    elements.</p>
+
+<ul><li>Case 5: removing multiple elements from an array</li></ul>
+<p>Fields to remove:</p>
+<pre>/addresses[0]
+/addresses[0]</pre>
+
+<p>In this example we want to remove the first two elements of the array.
+To do that we need to specify two separate path expressions, each pointing
+to one array element. Each removal is executed on the result of the previous
+removal, and removals are executed in the order in which the properties
+containing record paths are specified on the Processor.
+First, "/addresses[0]" is removed,
+that is the address with zip code 1111 in the example. After this removal,
+the addresses array has a new first element, which is the second element of
+the original array (the address with zip code 1121).
+To remove this element, we need to issue "/addresses[0]"
+again. Trying to remove "/addresses[0,1]", or filtering array elements
+with predicates when the target of the removal is multiple different array
+elements may produce unexpected results.</p>
+
+<ul><li>Case 6: array within an array</li></ul>
+<p>Let's suppose we have a complex input record that has an array within
+    an array.</p>
+
+<p>Input data:</p>
+<pre>{
+    "id": 1,
+    "people": [
+        {
+            "id": 11,
+            "addresses": [
+                {
+                    "zip": 1111,
+                    "street": "Main",
+                    "number": 24
+                },
+                {
+                    "zip": 1121,
+                    "street": "Airport",
+                    "number": 12
+                }
+            ]
+        },
+        {
+            "id": 22,
+            "addresses": [
+                {
+                    "zip": 2222,
+                    "street": "Ocean",
+                    "number": 24
+                },
+                {
+                    "zip": 2232,
+                    "street": "Sunset",
+                    "number": 12
+                }
+            ]
+        },
+        {
+            "id": 33,
+            "addresses": [
+                {
+                    "zip": 3333,
+                    "street": "Dawn",
+                    "number": 24
+                },
+                {
+                    "zip": 3323,
+                    "street": "Spring",
+                    "number": 12
+                }
+            ]
+        }
+    ]
+}</pre>
+
+<p>The following table summarizes what happens to the record and the schema for different RecordPaths</p>
+
+<table><tbody>
+<tr>
+    <th>Field To Remove</th>
+    <th>Is the schema modified?</th>
+    <th>What happens to the record?</th>
+</tr>
+<tr>
+    <td>/people[0]/addresses[1]/zip</td>
+    <td>No</td>
+    <td>The <i>zip</i> field of the first person's second address is set to <i>null</i>.</td>
+</tr>
+<tr>
+    <td>/people[*]/addresses[1]/zip</td>
+    <td>No</td>
+    <td>The <i>zip</i> field of all people's second address is set to <i>null</i>.</td>
+</tr>
+<tr>
+    <td>/people[0]/addresses[*]/zip</td>
+    <td>No</td>
+    <td>The <i>zip</i> field of the first person's every address is set to <i>null</i>.</td>
+</tr>
+<tr>
+    <td>/people[*]/addresses[*]/zip</td>
+    <td>Yes</td>
+    <td>The <i>zip</i> field every person's every address is removed (from the schema AND the data).</td>
+</tr>
+    <tbody></table>
+
+<p>The rules and examples shown for arrays apply for maps as well.</p>
+
+<h3><u>Example 4:</u></h3>
+<p>
+    <b>Choice datatype</b>
+</p>
+<p>Let's suppose we have an input schema that contains a field of type CHOICE.</p>
+<p>Input data:</p>
+<pre>{
+    "id": 12,
+    "name": "John Doe"
+}</pre>
+
+<p>Input schema:</p>
+<pre>{
+    "type": "record",
+    "name": "nameRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": [
+                "string",
+                {
+                    "type": "record",
+                    "name": "nameType",
+                    "fields": [
+                        {
+                            "name": "firstName",
+                            "type": "string"
+                        },
+                        {
+                            "name": "lastName",
+                            "type": "string"
+                        }
+                    ]
+                }
+            ]
+        }
+    ]
+}</pre>
+<p>In this example, the schema specifies the <i>name</i> field as CHOICE, but
+    in the data it is a simple string. If we remove "/name/firstName"
+    then there is no modifications to the data, but the schema is
+    modified, the <i>firstName</i> field gets removed from the schema only.</p>
+</body>
+</html>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java
new file mode 100644
index 0000000000..f02e8167f3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestRemoveRecordField.java
@@ -0,0 +1,399 @@
+/*
+ * 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.serialization.SchemaRegistryRecordSetWriter;
+import org.apache.nifi.util.MockFlowFile;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.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.junit.jupiter.api.Assertions.assertEquals;
+
+
+class TestRemoveRecordField {
+
+    private TestRunner runner;
+
+    private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper();
+
+    private static final String TEST_RESOURCES_FOLDER = "src/test/resources/TestRemoveRecordField";
+
+    @BeforeEach
+    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, JsonRecordSetWriter.SUPPRESS_NULLS, JsonRecordSetWriter.NEVER_SUPPRESS);
+        runner.setProperty(jsonWriter, JsonRecordSetWriter.PRETTY_PRINT_JSON, "true");
+        runner.setProperty(jsonWriter, jsonWriter.getSchemaWriteStrategyDescriptor(), SchemaRegistryRecordSetWriter.AVRO_SCHEMA_ATTRIBUTE);
+        runner.enableControllerService(jsonWriter);
+        runner.setProperty(AbstractRecordProcessor.RECORD_WRITER, "writer");
+    }
+
+    @Test
+    void testNotRootPath() throws InitializationException, IOException {
+        setUpJsonReader(null);
+
+        runner.setProperty("root_path", "/");
+        runner.assertNotValid();
+    }
+
+    @Test
+    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
+    void testRemoveSimpleFieldWithSchemaInference() throws InitializationException, IOException {
+        final String inputContent = "complex-person.json";
+        final String outputContent = "complex-person-no-dateOfBirth.json";
+        final String fieldToRemove = "/dateOfBirth";
+
+        executeRemovalTest(null, inputContent, null, outputContent, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveSimpleFieldThatIsMissingFromOneRecord() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputContent = "complex-person.json";
+        final String outputSchema = "complex-person-no-dateOfBirth.avsc";
+        final String outputContent = "complex-person-no-dateOfBirth.json";
+        final String fieldToRemove = "/dateOfBirth";
+
+        executeRemovalTest(inputSchema, inputContent, outputSchema, outputContent, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveComplexFieldThatIsMissingFromOneRecord() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person-no-workAddress.avsc";
+        final String outputFlowFile = "complex-person-no-workAddress.json";
+        final String fieldToRemove = "/workAddress";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveFieldFrom3LevelDeepStructure() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person-no-workAddress-building-letter.avsc";
+        final String outputFlowFile = "complex-person-no-workAddress-building-letter.json";
+        final String fieldToRemove = "/workAddress/building/letter";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveNestedFieldFromDeepStructureWithRelativePath() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person-no-workAddress-building-letter.avsc";
+        final String outputFlowFile = "complex-person-no-workAddress-building-letter.json";
+        final String fieldToRemove = "/workAddress//letter";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveFieldFromNonExistentParent() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person.avsc";
+        final String outputFlowFile = "complex-person.json";
+        final String fieldToRemove = "/workAddress/nonExistent/letter";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveNonExistentField() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person.avsc";
+        final String outputFlowFile = "complex-person.json";
+        final String fieldToRemove = "/nonExistentField";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveMultipleFields() throws InitializationException, IOException {
+        final String inputSchema = "complex-person.avsc";
+        final String inputFlowFile = "complex-person.json";
+        final String outputSchema = "complex-person-multiple-fields-removed.avsc";
+        final String outputFlowFile = "complex-person-multiple-fields-removed.json";
+        final String fieldToRemove1 = "/name";
+        final String fieldToRemove2 = "/dateOfBirth";
+        final String fieldToRemove3 = "/workAddress/building";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove1, fieldToRemove2, fieldToRemove3);
+    }
+
+    @Test
+    void testRemoveEntireArrayFieldWithSchemaInference() throws InitializationException, IOException {
+        final String inputFlowFile = "address-array.json";
+        final String outputFlowFile = "address-array-removed.json";
+        final String fieldToRemove = "/addresses";
+
+        executeRemovalTest(null, inputFlowFile, null, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveEntireArrayField() throws InitializationException, IOException {
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array-field-removed.avsc";
+        final String outputFlowFile = "address-array-removed.json";
+        final String fieldToRemove = "/addresses";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveOneElementFromArray() throws InitializationException, IOException {
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array.avsc";
+        final String outputFlowFile = "address-array-one-element-removed.json";
+        final String fieldToRemove = "/addresses[1]";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveAllElementsFromArrayAsterisk() throws InitializationException, IOException {
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array.avsc";
+        final String outputFlowFile = "address-array-empty.json";
+        final String fieldToRemove = "/addresses[*]";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveAllElementsFromArrayIndexRange() throws InitializationException, IOException {
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array.avsc";
+        final String outputFlowFile = "address-array-empty.json";
+        final String fieldToRemove = "/addresses[0..-1]";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveFieldFromOneArrayElement() throws InitializationException, IOException {
+        // The schema must not be modified.
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array.avsc";
+        final String outputFlowFile = "address-array-zip-removed-from-one-element.json";
+        final String fieldToRemove = "/addresses[1]/zip";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testRemoveFieldFromAllArrayElements() throws InitializationException, IOException {
+        // The schema must be modified.
+        final String inputSchema = "address-array.avsc";
+        final String inputFlowFile = "address-array.json";
+        final String outputSchema = "address-array-zip-removed.avsc";
+        final String outputFlowFile = "address-array-zip-removed-from-all-elements.json";
+        final String fieldToRemove = "/addresses[*]/zip";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testMapRemoveOneItem() throws InitializationException, IOException {
+        // The schema must not be modified.
+        final String inputSchema = "bookshelf-map.avsc";
+        final String inputFlowFile = "bookshelf-map.json";
+        final String outputSchema = "bookshelf-map.avsc";
+        final String outputFlowFile = "bookshelf-map-one-item-removed.json";
+        final String fieldToRemove = "/books['ISBN_2222222222222']";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testMapRemoveFieldFromOneItem() throws InitializationException, IOException {
+        // The schema must not be modified.
+        final String inputSchema = "bookshelf-map.avsc";
+        final String inputFlowFile = "bookshelf-map.json";
+        final String outputSchema = "bookshelf-map.avsc";
+        final String outputFlowFile = "bookshelf-map-field-removed-from-one-element.json";
+        final String fieldToRemove = "/books['ISBN_2222222222222']/author";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testMapRemoveFieldFromAllItems() throws InitializationException, IOException {
+        // The schema must be modified.
+        final String inputSchema = "bookshelf-map.avsc";
+        final String inputFlowFile = "bookshelf-map.json";
+        final String outputSchema = "bookshelf-map-author-field-removed.avsc";
+        final String outputFlowFile = "bookshelf-map-field-removed-from-all-elements.json";
+        final String fieldToRemove = "/books[*]/author";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testMapRemoveAllItems() throws InitializationException, IOException {
+        // The schema must not be modified.
+        final String inputSchema = "bookshelf-map.avsc";
+        final String inputFlowFile = "bookshelf-map.json";
+        final String outputSchema = "bookshelf-map.avsc";
+        final String outputFlowFile = "bookshelf-map-all-items-removed.json";
+        final String fieldToRemove = "/books[*]";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testChoiceRemoveFieldFromPossibleSubtypeWithSchemaInference() throws InitializationException, IOException {
+        final String inputFlowFile = "name-choice.json";
+        final String outputFlowFile = "name-choice-firstName-removed.json";
+        final String fieldToRemove = "/name/firstName";
+
+        executeRemovalTest(null, inputFlowFile, null, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testChoiceRemoveFieldFromPossibleSubtype() throws InitializationException, IOException {
+        // The schema must be modified.
+        final String inputSchema = "name-choice.avsc";
+        final String inputFlowFile = "name-choice.json";
+        final String outputSchema = "name-choice-firstName-removed.avsc";
+        final String outputFlowFile = "name-choice-firstName-removed.json";
+        final String fieldToRemove = "/name/firstName";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    @Test
+    void testChoiceRemoveFieldFromChoiceInChoice() throws InitializationException, IOException {
+        // The schema must be modified.
+        final String inputSchema = "address-choice.avsc";
+        final String inputFlowFile = "address-choice.json";
+        final String outputSchema = "address-choice-letter-removed-from-building.avsc";
+        final String outputFlowFile = "address-choice-letter-removed-from-building.json";
+        final String fieldToRemove = "/address/building/letter";
+
+        executeRemovalTest(inputSchema, inputFlowFile, outputSchema, outputFlowFile, fieldToRemove);
+    }
+
+    private void executeRemovalTest(final String inputSchema, final String inputFlowFile, final String outputSchema, final String outputFlowFile, final String... fieldsToRemove)
+            throws IOException, InitializationException {
+        executePreparation(inputSchema, inputFlowFile, fieldsToRemove);
+
+        runner.run();
+
+        assertOutput(outputSchema, outputFlowFile);
+    }
+
+    private void executePreparation(final String inputSchema, final String inputFlowFile, final String... fieldsToRemove) throws IOException, InitializationException {
+        setUpJsonReader(inputSchema);
+
+        final String fieldName = "field-to-remove-";
+        final AtomicInteger counter = new AtomicInteger(0);
+        final Map<String, String> properties = Arrays.stream(fieldsToRemove)
+                .collect(Collectors.toMap(f -> fieldName + counter.incrementAndGet(), f -> f));
+        setUpRunner(inputFlowFile, properties);
+    }
+
+    private void assertOutput(final String outputSchema, final String outputContent) throws IOException {
+        runner.assertAllFlowFilesTransferred(AbstractRecordProcessor.REL_SUCCESS, 1);
+        final MockFlowFile out = runner.getFlowFilesForRelationship(AbstractRecordProcessor.REL_SUCCESS).get(0);
+
+        final JsonNode expectedOutput = OBJECT_MAPPER.readTree(Paths.get(TEST_RESOURCES_FOLDER, "output", outputContent).toFile());
+        final JsonNode output = OBJECT_MAPPER.readTree(out.getContent());
+        assertEquals(expectedOutput, output);
+
+        if (outputSchema != null) {
+            final JsonNode expectedSchema = OBJECT_MAPPER.readTree(Paths.get(TEST_RESOURCES_FOLDER, "output_schema", outputSchema).toFile());
+            final JsonNode schema = OBJECT_MAPPER.readTree(out.getAttribute("avro.schema"));
+            assertEquals(expectedSchema, schema);
+        } else {
+            out.assertAttributeExists("avro.schema");
+            out.assertAttributeNotEquals("avro.schema", "");
+        }
+    }
+
+    private void setUpJsonReader(final String schemaFilePath) throws IOException, InitializationException {
+        final JsonTreeReader jsonReader = new JsonTreeReader();
+        runner.addControllerService("reader", jsonReader);
+        if (schemaFilePath == null) {
+            runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaInferenceUtil.INFER_SCHEMA);
+        } else {
+            final String inputSchemaText = new String(Files.readAllBytes(Paths.get(TEST_RESOURCES_FOLDER, "input_schema", schemaFilePath)));
+            runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaAccessUtils.SCHEMA_TEXT_PROPERTY);
+            runner.setProperty(jsonReader, SchemaAccessUtils.SCHEMA_TEXT, inputSchemaText);
+        }
+        runner.enableControllerService(jsonReader);
+        runner.setProperty(AbstractRecordProcessor.RECORD_READER, "reader");
+    }
+
+    private void setUpRunner(final String flowFilePath, final Map<String, String> properties) throws IOException {
+        runner.enqueue(Paths.get(TEST_RESOURCES_FOLDER, "input", flowFilePath));
+        properties.forEach((propertyName,propertyValue) -> runner.setProperty(propertyName, propertyValue));
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-array.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-array.json
new file mode 100644
index 0000000000..97f64e1d4f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-array.json
@@ -0,0 +1,30 @@
+[ {
+  "id" : 11,
+  "name" : "John",
+  "addresses" : [
+    {
+      "zip" : 1111,
+      "street" : "Main",
+      "building" : {
+        "number" : 1,
+        "letter" : "A"
+      }
+    },
+    {
+      "zip" : 2222,
+      "street" : "Ocean",
+      "building" : {
+        "number" : 2,
+        "letter" : "B"
+      }
+    },
+    {
+      "zip" : 3333,
+      "street" : "Sunset",
+      "building" : {
+        "number" : 3,
+        "letter" : "C"
+      }
+    }
+  ]
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-choice.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-choice.json
new file mode 100644
index 0000000000..1a0ae53cc9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/address-choice.json
@@ -0,0 +1,25 @@
+[
+  {
+    "id": 11,
+    "address": "11 Main Street, Aville"
+  },
+  {
+    "id": 22,
+    "address": {
+      "zip": 2222,
+      "street": "Turner Street",
+      "building": 222
+    }
+  },
+  {
+    "id": 33,
+    "address": {
+      "zip": 3333,
+      "street": "West Street",
+      "building": {
+        "number": 333,
+        "letter": "C"
+      }
+    }
+  }
+]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/bookshelf-map.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/bookshelf-map.json
new file mode 100644
index 0000000000..75abe7d944
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/bookshelf-map.json
@@ -0,0 +1,16 @@
+[ {
+  "books" : {
+    "ISBN_1111111111111" : {
+      "author" : "Author 1",
+      "title" : "Title 1"
+    },
+    "ISBN_2222222222222" : {
+      "author" : "Author 2",
+      "title" : "Title 2"
+    },
+    "ISBN_3333333333333" : {
+      "author" : "Author 3",
+      "title" : "Title 3"
+    }
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/complex-person.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/complex-person.json
new file mode 100644
index 0000000000..f825982057
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/complex-person.json
@@ -0,0 +1,29 @@
+[ {
+  "id": 17,
+  "name": "John",
+  "dateOfBirth": "1980-01-01",
+  "homeAddress": {
+    "zip": 1111,
+    "street": "Main",
+    "building": {
+      "number": 24,
+      "letter": "A"
+    }
+  },
+  "workAddress": {
+    "zip": 2222,
+    "street": "Airport",
+    "building": {
+      "number": 24,
+      "letter": "A"
+    }
+  }
+},
+  {
+    "id": 28,
+    "name": "Bob",
+    "homeAddress": {
+      "zip": 3333,
+      "street": "Atlantic"
+    }
+  } ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/name-choice.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/name-choice.json
new file mode 100644
index 0000000000..75092304b9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input/name-choice.json
@@ -0,0 +1,10 @@
+[ {
+  "id" : 11,
+  "name" : {
+    "firstName" : "John",
+    "lastName" : "Doe"
+  }
+}, {
+  "id" : 12,
+  "name" : "Robert Smith"
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-array.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-array.avsc
new file mode 100644
index 0000000000..d3957c1b2c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-array.avsc
@@ -0,0 +1,52 @@
+{
+    "type": "record",
+    "name": "nifiRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": "string"
+        },
+        {
+            "name": "addresses",
+            "type": {
+                "type": "array",
+                "items": {
+                    "name": "address",
+                    "type": "record",
+                    "fields": [
+                        {
+                            "name": "zip",
+                            "type": "int"
+                        },
+                        {
+                            "name": "street",
+                            "type": "string"
+                        },
+                        {
+                            "name": "building",
+                            "type": {
+                                "name": "buildingID",
+                                "type": "record",
+                                "fields": [
+                                    {
+                                        "name": "number",
+                                        "type": "int"
+                                    },
+                                    {
+                                        "name": "letter",
+                                        "type": "string"
+                                    }
+                                ]
+                            }
+                        }
+                    ]
+                }
+            }
+        }
+    ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-choice.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-choice.avsc
new file mode 100644
index 0000000000..550da229ac
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/address-choice.avsc
@@ -0,0 +1,51 @@
+{
+    "type": "record",
+    "name": "addressRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "address",
+            "type": [
+                "string",
+                {
+                    "type": "record",
+                    "name": "addressType",
+                    "fields": [
+                        {
+                            "name": "zip",
+                            "type": "int"
+                        },
+                        {
+                            "name": "street",
+                            "type": "string"
+                        },
+                        {
+                            "name": "building",
+                            "type": [
+                                "int",
+                                {
+                                    "type": "record",
+                                    "name": "buildingType",
+                                    "fields": [
+                                        {
+                                            "name": "number",
+                                            "type": "int"
+                                        },
+                                        {
+                                            "name": "letter",
+                                            "type": "string"
+                                        }
+                                    ]
+                                }
+                            ]
+                        }
+                    ]
+                }
+            ]
+        }
+    ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/bookshelf-map.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/bookshelf-map.avsc
new file mode 100644
index 0000000000..fcc6f9966a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/bookshelf-map.avsc
@@ -0,0 +1,27 @@
+{
+    "type": "record",
+    "name": "bookshelf",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "books",
+            "type": {
+                "type": "map",
+                "values": {
+                    "type": "record",
+                    "name": "bookType",
+                    "fields": [
+                        {
+                            "name": "author",
+                            "type": "string"
+                        },
+                        {
+                            "name": "title",
+                            "type": "string"
+                        }
+                    ]
+                }
+            }
+        }
+    ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/complex-person.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/complex-person.avsc
new file mode 100644
index 0000000000..d487fdc898
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/complex-person.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"dateOfBirth","type":"string"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}},{"name":"workAdd [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/name-choice.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/name-choice.avsc
new file mode 100644
index 0000000000..25881931ea
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/input_schema/name-choice.avsc
@@ -0,0 +1,31 @@
+{
+    "type": "record",
+    "name": "nameRecord",
+    "namespace": "org.apache.nifi",
+    "fields": [
+        {
+            "name": "id",
+            "type": "int"
+        },
+        {
+            "name": "name",
+            "type": [
+                "string",
+                {
+                    "type": "record",
+                    "name": "nameType",
+                    "fields": [
+                        {
+                            "name": "firstName",
+                            "type": "string"
+                        },
+                        {
+                            "name": "lastName",
+                            "type": "string"
+                        }
+                    ]
+                }
+            ]
+        }
+    ]
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-empty.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-empty.json
new file mode 100644
index 0000000000..bcef8abe87
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-empty.json
@@ -0,0 +1,5 @@
+[ {
+  "id" : 11,
+  "name" : "John",
+  "addresses" : [ ]
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-one-element-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-one-element-removed.json
new file mode 100644
index 0000000000..4ebe1026b3
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-one-element-removed.json
@@ -0,0 +1,19 @@
+[ {
+  "id" : 11,
+  "name" : "John",
+  "addresses" : [ {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 1,
+      "letter" : "A"
+    }
+  }, {
+    "zip" : 3333,
+    "street" : "Sunset",
+    "building" : {
+      "number" : 3,
+      "letter" : "C"
+    }
+  } ]
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-removed.json
new file mode 100644
index 0000000000..394ebbfbeb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-removed.json
@@ -0,0 +1,4 @@
+[ {
+  "id" : 11,
+  "name" : "John"
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-all-elements.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-all-elements.json
new file mode 100644
index 0000000000..1c59843d3b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-all-elements.json
@@ -0,0 +1,23 @@
+[ {
+  "id" : 11,
+  "name" : "John",
+  "addresses" : [ {
+    "street" : "Main",
+    "building" : {
+      "number" : 1,
+      "letter" : "A"
+    }
+  }, {
+    "street" : "Ocean",
+    "building" : {
+      "number" : 2,
+      "letter" : "B"
+    }
+  }, {
+    "street" : "Sunset",
+    "building" : {
+      "number" : 3,
+      "letter" : "C"
+    }
+  } ]
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-one-element.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-one-element.json
new file mode 100644
index 0000000000..676a5a6234
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-array-zip-removed-from-one-element.json
@@ -0,0 +1,26 @@
+[ {
+  "id" : 11,
+  "name" : "John",
+  "addresses" : [ {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 1,
+      "letter" : "A"
+    }
+  }, {
+    "zip" : null,
+    "street" : "Ocean",
+    "building" : {
+      "number" : 2,
+      "letter" : "B"
+    }
+  }, {
+    "zip" : 3333,
+    "street" : "Sunset",
+    "building" : {
+      "number" : 3,
+      "letter" : "C"
+    }
+  } ]
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-choice-letter-removed-from-building.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-choice-letter-removed-from-building.json
new file mode 100644
index 0000000000..f6ab258e7d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/address-choice-letter-removed-from-building.json
@@ -0,0 +1,20 @@
+[ {
+  "id" : 11,
+  "address" : "11 Main Street, Aville"
+}, {
+  "id" : 22,
+  "address" : {
+    "zip" : 2222,
+    "street" : "Turner Street",
+    "building" : 222
+  }
+}, {
+  "id" : 33,
+  "address" : {
+    "zip" : 3333,
+    "street" : "West Street",
+    "building" : {
+      "number" : 333
+    }
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-all-items-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-all-items-removed.json
new file mode 100644
index 0000000000..6ccde6f74c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-all-items-removed.json
@@ -0,0 +1,3 @@
+[ {
+  "books" : { }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-all-elements.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-all-elements.json
new file mode 100644
index 0000000000..3392f8fc75
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-all-elements.json
@@ -0,0 +1,13 @@
+[ {
+  "books" : {
+    "ISBN_1111111111111" : {
+      "title" : "Title 1"
+    },
+    "ISBN_2222222222222" : {
+      "title" : "Title 2"
+    },
+    "ISBN_3333333333333" : {
+      "title" : "Title 3"
+    }
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-one-element.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-one-element.json
new file mode 100644
index 0000000000..5a8e057e66
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-field-removed-from-one-element.json
@@ -0,0 +1,16 @@
+[ {
+  "books" : {
+    "ISBN_1111111111111" : {
+      "author" : "Author 1",
+      "title" : "Title 1"
+    },
+    "ISBN_2222222222222" : {
+      "author" : null,
+      "title" : "Title 2"
+    },
+    "ISBN_3333333333333" : {
+      "author" : "Author 3",
+      "title" : "Title 3"
+    }
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-one-item-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-one-item-removed.json
new file mode 100644
index 0000000000..65da4ffcd9
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/bookshelf-map-one-item-removed.json
@@ -0,0 +1,12 @@
+[ {
+  "books" : {
+    "ISBN_1111111111111" : {
+      "author" : "Author 1",
+      "title" : "Title 1"
+    },
+    "ISBN_3333333333333" : {
+      "author" : "Author 3",
+      "title" : "Title 3"
+    }
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-multiple-fields-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-multiple-fields-removed.json
new file mode 100644
index 0000000000..17a9e86c2a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-multiple-fields-removed.json
@@ -0,0 +1,23 @@
+[ {
+  "id" : 17,
+  "homeAddress" : {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  },
+  "workAddress" : {
+    "zip" : 2222,
+    "street" : "Airport"
+  }
+}, {
+  "id" : 28,
+  "homeAddress" : {
+    "zip" : 3333,
+    "street" : "Atlantic",
+    "building" : null
+  },
+  "workAddress" : null
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-dateOfBirth.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-dateOfBirth.json
new file mode 100644
index 0000000000..2e4a267eed
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-dateOfBirth.json
@@ -0,0 +1,29 @@
+[ {
+  "id" : 17,
+  "name" : "John",
+  "homeAddress" : {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  },
+  "workAddress" : {
+    "zip" : 2222,
+    "street" : "Airport",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  }
+}, {
+  "id" : 28,
+  "name" : "Bob",
+  "homeAddress" : {
+    "zip" : 3333,
+    "street" : "Atlantic",
+    "building" : null
+  },
+  "workAddress" : null
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress-building-letter.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress-building-letter.json
new file mode 100644
index 0000000000..3d0cc15ccb
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress-building-letter.json
@@ -0,0 +1,30 @@
+[ {
+  "id" : 17,
+  "name" : "John",
+  "dateOfBirth" : "1980-01-01",
+  "homeAddress" : {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  },
+  "workAddress" : {
+    "zip" : 2222,
+    "street" : "Airport",
+    "building" : {
+      "number" : 24
+    }
+  }
+}, {
+  "id" : 28,
+  "name" : "Bob",
+  "dateOfBirth" : null,
+  "homeAddress" : {
+    "zip" : 3333,
+    "street" : "Atlantic",
+    "building" : null
+  },
+  "workAddress" : null
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress.json
new file mode 100644
index 0000000000..90d6d701dc
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person-no-workAddress.json
@@ -0,0 +1,22 @@
+[ {
+  "id" : 17,
+  "name" : "John",
+  "dateOfBirth" : "1980-01-01",
+  "homeAddress" : {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  }
+}, {
+  "id" : 28,
+  "name" : "Bob",
+  "dateOfBirth" : null,
+  "homeAddress" : {
+    "zip" : 3333,
+    "street" : "Atlantic",
+    "building" : null
+  }
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person.json
new file mode 100644
index 0000000000..ab1227da6b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/complex-person.json
@@ -0,0 +1,31 @@
+[ {
+  "id" : 17,
+  "name" : "John",
+  "dateOfBirth" : "1980-01-01",
+  "homeAddress" : {
+    "zip" : 1111,
+    "street" : "Main",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  },
+  "workAddress" : {
+    "zip" : 2222,
+    "street" : "Airport",
+    "building" : {
+      "number" : 24,
+      "letter" : "A"
+    }
+  }
+}, {
+  "id" : 28,
+  "name" : "Bob",
+  "dateOfBirth" : null,
+  "homeAddress" : {
+    "zip" : 3333,
+    "street" : "Atlantic",
+    "building" : null
+  },
+  "workAddress" : null
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/name-choice-firstName-removed.json b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/name-choice-firstName-removed.json
new file mode 100644
index 0000000000..a16eb83f6a
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output/name-choice-firstName-removed.json
@@ -0,0 +1,9 @@
+[ {
+  "id" : 11,
+  "name" : {
+    "lastName" : "Doe"
+  }
+}, {
+  "id" : 12,
+  "name" : "Robert Smith"
+} ]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-field-removed.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-field-removed.avsc
new file mode 100644
index 0000000000..1506cbcb3c
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-field-removed.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-zip-removed.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-zip-removed.avsc
new file mode 100644
index 0000000000..53b7f170fa
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array-zip-removed.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"addresses","type":{"type":"array","items":{"type":"record","name":"addressesType","fields":[{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"addresses_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}}}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array.avsc
new file mode 100644
index 0000000000..52da7a170e
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-array.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"addresses","type":{"type":"array","items":{"type":"record","name":"addressesType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"addresses_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}}}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-choice-letter-removed-from-building.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-choice-letter-removed-from-building.avsc
new file mode 100644
index 0000000000..09ba6dee05
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/address-choice-letter-removed-from-building.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"address","type":["string",{"type":"record","name":"addressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":["int",{"type":"record","name":"address_buildingType","fields":[{"name":"number","type":"int"}]}]}]}]}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map-author-field-removed.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map-author-field-removed.avsc
new file mode 100644
index 0000000000..56bd070933
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map-author-field-removed.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"books","type":{"type":"map","values":{"type":"record","name":"booksType","fields":[{"name":"title","type":"string"}]}}}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map.avsc
new file mode 100644
index 0000000000..3b084b88a8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/bookshelf-map.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"books","type":{"type":"map","values":{"type":"record","name":"booksType","fields":[{"name":"author","type":"string"},{"name":"title","type":"string"}]}}}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-multiple-fields-removed.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-multiple-fields-removed.avsc
new file mode 100644
index 0000000000..9135fdb44b
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-multiple-fields-removed.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}},{"name":"workAddress","type":{"type":"record","name":"workAddressType","fields":[{"name [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-dateOfBirth.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-dateOfBirth.avsc
new file mode 100644
index 0000000000..98bdc60a7f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-dateOfBirth.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}},{"name":"workAddress","type":{"type":"record","name":"w [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress-building-letter.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress-building-letter.avsc
new file mode 100644
index 0000000000..4fd076d595
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress-building-letter.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"dateOfBirth","type":"string"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}},{"name":"workAdd [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress.avsc
new file mode 100644
index 0000000000..b6ff2ccba8
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person-no-workAddress.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"dateOfBirth","type":"string"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person.avsc
new file mode 100644
index 0000000000..d487fdc898
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/complex-person.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":"string"},{"name":"dateOfBirth","type":"string"},{"name":"homeAddress","type":{"type":"record","name":"homeAddressType","fields":[{"name":"zip","type":"int"},{"name":"street","type":"string"},{"name":"building","type":{"type":"record","name":"homeAddress_buildingType","fields":[{"name":"number","type":"int"},{"name":"letter","type":"string"}]}}]}},{"name":"workAdd [...]
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/name-choice-firstName-removed.avsc b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/name-choice-firstName-removed.avsc
new file mode 100644
index 0000000000..37a940867f
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/resources/TestRemoveRecordField/output_schema/name-choice-firstName-removed.avsc
@@ -0,0 +1 @@
+{"type":"record","name":"nifiRecord","namespace":"org.apache.nifi","fields":[{"name":"id","type":"int"},{"name":"name","type":["string",{"type":"record","name":"nameType","fields":[{"name":"lastName","type":"string"}]}]}]}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
index 1f7e16c890..83547e890d 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonRecordSetWriter.java
@@ -19,6 +19,7 @@ package org.apache.nifi.json;
 
 import org.apache.commons.compress.compressors.CompressorException;
 import org.apache.commons.compress.compressors.CompressorStreamFactory;
+import org.apache.nifi.NullSuppression;
 import org.apache.nifi.annotation.documentation.CapabilityDescription;
 import org.apache.nifi.annotation.documentation.Tags;
 import org.apache.nifi.annotation.lifecycle.OnEnabled;
@@ -29,7 +30,6 @@ import org.apache.nifi.components.ValidationResult;
 import org.apache.nifi.controller.ConfigurationContext;
 import org.apache.nifi.expression.ExpressionLanguageScope;
 import org.apache.nifi.logging.ComponentLog;
-import org.apache.nifi.NullSuppression;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.serialization.DateTimeTextRecordSetWriter;
 import org.apache.nifi.serialization.RecordSetWriter;
@@ -54,16 +54,16 @@ import java.util.Map;
         + "consists of a single row, it will be written as an array with a single element. If using One Line Per Object output, the JSON objects cannot be pretty-printed.")
 public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements RecordSetWriterFactory {
 
-    static final AllowableValue ALWAYS_SUPPRESS = new AllowableValue("always-suppress", "Always Suppress",
+    public static final AllowableValue ALWAYS_SUPPRESS = new AllowableValue("always-suppress", "Always Suppress",
             "Fields that are missing (present in the schema but not in the record), or that have a value of null, will not be written out");
-    static final AllowableValue NEVER_SUPPRESS = new AllowableValue("never-suppress", "Never Suppress",
+    public static final AllowableValue NEVER_SUPPRESS = new AllowableValue("never-suppress", "Never Suppress",
             "Fields that are missing (present in the schema but not in the record), or that have a value of null, will be written out as a null value");
-    static final AllowableValue SUPPRESS_MISSING = new AllowableValue("suppress-missing", "Suppress Missing Values",
+    public static final AllowableValue SUPPRESS_MISSING = new AllowableValue("suppress-missing", "Suppress Missing Values",
             "When a field has a value of null, it will be written out. However, if a field is defined in the schema and not present in the record, the field will not be written out.");
 
-    static final AllowableValue OUTPUT_ARRAY = new AllowableValue("output-array", "Array",
+    public static final AllowableValue OUTPUT_ARRAY = new AllowableValue("output-array", "Array",
             "Output records as a JSON array");
-    static final AllowableValue OUTPUT_ONELINE = new AllowableValue("output-oneline", "One Line Per Object",
+    public static final AllowableValue OUTPUT_ONELINE = new AllowableValue("output-oneline", "One Line Per Object",
             "Output records with one JSON object per line, delimited by a newline character");
 
     public static final String COMPRESSION_FORMAT_GZIP = "gzip";
@@ -74,7 +74,7 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
     public static final String COMPRESSION_FORMAT_NONE = "none";
     public static final String COMPRESSION_FORMAT_ZSTD = "zstd";
 
-    static final PropertyDescriptor SUPPRESS_NULLS = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor SUPPRESS_NULLS = new PropertyDescriptor.Builder()
             .name("suppress-nulls")
             .displayName("Suppress Null Values")
             .description("Specifies how the writer should handle a null field")
@@ -82,7 +82,7 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
             .defaultValue(NEVER_SUPPRESS.getValue())
             .required(true)
             .build();
-    static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor PRETTY_PRINT_JSON = new PropertyDescriptor.Builder()
             .name("Pretty Print JSON")
             .description("Specifies whether or not the JSON should be pretty printed")
             .expressionLanguageSupported(ExpressionLanguageScope.NONE)
@@ -90,7 +90,7 @@ public class JsonRecordSetWriter extends DateTimeTextRecordSetWriter implements
             .defaultValue("false")
             .required(true)
             .build();
-    static final PropertyDescriptor OUTPUT_GROUPING = new PropertyDescriptor.Builder()
+    public static final PropertyDescriptor OUTPUT_GROUPING = new PropertyDescriptor.Builder()
             .name("output-grouping")
             .displayName("Output Grouping")
             .description("Specifies how the writer should output the JSON records (as an array or one object per line, e.g.) Note that if 'One Line Per Object' is "