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 "