From 80fccb323d1c8cbde442dc96ef35226d1121c9b8 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 19 Apr 2017 12:39:35 -0400 Subject: [PATCH 1/8] NIFI-3682: Add Schema Access Strategy and Schema Write Strategy Record Readers and Writers; bug fixes. --- .../apache/nifi/components/PropertyValue.java | 9 + .../language/EmptyPreparedQuery.java | 5 + .../language/InvalidPreparedQuery.java | 5 + .../expression/language/PreparedQuery.java | 2 + .../language/StandardPreparedQuery.java | 5 + .../language/StandardPropertyValue.java | 5 + .../apache/nifi/util/MockPropertyValue.java | 13 ++ .../scheduling/ConnectableProcessContext.java | 5 + .../src/main/resources/conf/logback.xml | 2 + .../services/AvroSchemaRegistry.java | 76 +++---- .../services/TestAvroSchemaRegistry.java | 19 +- .../{QueryFlowFile.java => QueryRecord.java} | 139 ++++++++----- .../FlowFileEnumerator.java | 39 ++-- .../FlowFileProjectTableScanRule.java | 2 +- .../FlowFileTable.java | 31 ++- .../FlowFileTableScan.java | 2 +- .../org.apache.nifi.processor.Processor | 2 +- .../additionalDetails.html | 4 +- ...ueryFlowFile.java => TestQueryRecord.java} | 36 ++-- .../util/record/MockRecordParser.java | 9 +- .../util/record/MockRecordWriter.java | 4 +- .../access/SchemaNotFoundException.java | 32 +++ ...rFactory.java => RecordReaderFactory.java} | 7 +- .../serialization/RecordSetWriterFactory.java | 37 +++- .../serialization/SimpleRecordSchema.java | 49 ++++- .../nifi/serialization/record/DataType.java | 12 +- .../nifi/serialization/record/MapRecord.java | 16 +- .../serialization/record/RecordFieldType.java | 53 ++++- .../serialization/record/RecordSchema.java | 15 ++ .../record/ResultSetRecordSet.java | 14 ++ .../record/SchemaIdentifier.java | 96 +++++++++ .../record/type/ArrayDataType.java | 2 +- .../record/type/ChoiceDataType.java | 2 +- .../record/type/MapDataType.java | 67 ++++++ .../record/type/RecordDataType.java | 5 + .../record/util/DataTypeUtils.java | 168 ++++++++++----- .../pom.xml | 7 +- .../java/org/apache/nifi/avro/AvroReader.java | 34 +++- .../avro/AvroReaderWithEmbeddedSchema.java | 62 ++++++ .../avro/AvroReaderWithExplicitSchema.java | 75 +++++++ .../apache/nifi/avro/AvroRecordReader.java | 41 +--- .../apache/nifi/avro/AvroRecordSetWriter.java | 66 ++++-- .../apache/nifi/avro/AvroSchemaValidator.java | 9 + .../org/apache/nifi/avro/AvroTypeUtil.java | 33 ++- .../org/apache/nifi/avro/WriteAvroResult.java | 59 ++---- .../WriteAvroResultWithExternalSchema.java | 75 +++++++ .../nifi/avro/WriteAvroResultWithSchema.java | 62 ++++++ .../nifi/csv/CSVHeaderSchemaStrategy.java | 59 ++++++ .../java/org/apache/nifi/csv/CSVReader.java | 43 +++- .../org/apache/nifi/csv/CSVRecordReader.java | 7 +- .../apache/nifi/csv/CSVRecordSetWriter.java | 11 +- .../java/org/apache/nifi/csv/CSVUtils.java | 1 + .../org/apache/nifi/csv/WriteCSVResult.java | 21 +- .../java/org/apache/nifi/grok/GrokReader.java | 115 +++++++++-- .../apache/nifi/grok/GrokRecordReader.java | 106 +--------- .../json/AbstractJsonRowRecordReader.java | 57 ------ .../org/apache/nifi/json/JsonPathReader.java | 11 +- .../nifi/json/JsonPathRowRecordReader.java | 10 +- .../apache/nifi/json/JsonRecordSetWriter.java | 10 +- .../org/apache/nifi/json/JsonTreeReader.java | 11 +- .../nifi/json/JsonTreeRowRecordReader.java | 70 +++++-- .../org/apache/nifi/json/WriteJsonResult.java | 73 +++++-- .../schema/access/AvroSchemaTextStrategy.java | 56 +++++ ...onworksEncodedSchemaReferenceStrategy.java | 60 ++++++ ...rtonworksEncodedSchemaReferenceWriter.java | 60 ++++++ .../schema/access/SchemaAccessStrategy.java | 29 +++ .../schema/access/SchemaAccessWriter.java | 47 +++++ .../schema/access/SchemaNameAsAttribute.java | 46 +++++ .../access/SchemaNamePropertyStrategy.java | 55 +++++ .../schema/access/SchemaTextAsAttribute.java | 46 +++++ .../DateTimeTextRecordSetWriter.java | 11 +- .../SchemaRegistryRecordReader.java | 110 ---------- .../SchemaRegistryRecordSetWriter.java | 113 +++++++++++ .../serialization/SchemaRegistryService.java | 192 ++++++++++++++++++ .../text/FreeFormTextRecordSetWriter.java | 9 +- ... => TestAvroReaderWithEmbeddedSchema.java} | 24 +-- .../apache/nifi/avro/TestWriteAvroResult.java | 37 ++-- .../avro/TestWriteAvroResultWithSchema.java | 46 +++++ .../TestWriteAvroResultWithoutSchema.java | 56 +++++ .../nifi/csv/TestCSVHeaderSchemaStrategy.java | 69 +++++++ .../apache/nifi/csv/TestWriteCSVResult.java | 7 +- .../nifi/grok/TestGrokRecordReader.java | 10 +- .../json/TestJsonTreeRowRecordReader.java | 23 +++ .../apache/nifi/json/TestWriteJsonResult.java | 14 +- .../test/resources/json/output/dataTypes.json | 6 +- .../single-bank-account-wrong-field-type.json | 13 ++ .../services/SchemaRegistry.java | 52 +++-- 87 files changed, 2545 insertions(+), 763 deletions(-) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/{QueryFlowFile.java => QueryRecord.java} (82%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/{queryflowfile => queryrecord}/FlowFileEnumerator.java (82%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/{queryflowfile => queryrecord}/FlowFileProjectTableScanRule.java (98%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/{queryflowfile => queryrecord}/FlowFileTable.java (88%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/{queryflowfile => queryrecord}/FlowFileTableScan.java (98%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/{org.apache.nifi.processors.standard.QueryFlowFile => org.apache.nifi.processors.standard.QueryRecord}/additionalDetails.html (94%) rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/{TestQueryFlowFile.java => TestQueryRecord.java} (91%) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java rename nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/{RowRecordReaderFactory.java => RecordReaderFactory.java} (84%) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java delete mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java rename nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/{TestAvroRecordReader.java => TestAvroReaderWithEmbeddedSchema.java} (94%) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json diff --git a/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java b/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java index efe76ee591b9..05f262fcf601 100644 --- a/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java +++ b/nifi-api/src/main/java/org/apache/nifi/components/PropertyValue.java @@ -324,4 +324,13 @@ public PropertyValue evaluateAttributeExpressions(FlowFile flowFile, Map + * Indicates whether the value of the property uses Expression Language. + *

+ * + * @return true if the property value makes use of the Expression Language, false otherwise. + */ + boolean isExpressionLanguagePresent(); } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java index 403753163f60..a435b0816763 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/EmptyPreparedQuery.java @@ -38,4 +38,9 @@ public String evaluateExpressions(Map valueLookup, AttributeValu public String evaluateExpressions(Map attributes, AttributeValueDecorator decorator, Map stateVariables) throws ProcessException { return value; } + + @Override + public boolean isExpressionLanguagePresent() { + return false; + } } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java index 1033c713eb9a..ce0dec38d4eb 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/InvalidPreparedQuery.java @@ -47,4 +47,9 @@ public String evaluateExpressions(final Map valueLookup, final At public String evaluateExpressions( Map valueLookup, AttributeValueDecorator decorator, Map stateVariables) throws ProcessException { throw new AttributeExpressionLanguageException("Invalid Expression: " + query + " due to " + explanation); } + + @Override + public boolean isExpressionLanguagePresent() { + return false; + } } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java index 53f7296234cb..5552cace69d6 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/PreparedQuery.java @@ -26,4 +26,6 @@ public interface PreparedQuery { String evaluateExpressions(Map valueLookup, AttributeValueDecorator decorator) throws ProcessException; String evaluateExpressions(final Map valueLookup, final AttributeValueDecorator decorator, final Map stateVariables) throws ProcessException; + + boolean isExpressionLanguagePresent(); } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java index 39cfb25f9f70..9f12c92447de 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java @@ -59,4 +59,9 @@ public String evaluateExpressions(final Map valMap, final Attrib throws ProcessException { return evaluateExpressions(valMap, decorator, null); } + + @Override + public boolean isExpressionLanguagePresent() { + return !trees.isEmpty(); + } } diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java index 94c1c5021e2f..ac370bd63180 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPropertyValue.java @@ -195,4 +195,9 @@ public T asControllerService(final Class servic public boolean isSet() { return rawValue != null; } + + @Override + public boolean isExpressionLanguagePresent() { + return preparedQuery.isExpressionLanguagePresent(); + } } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java index b6752a758864..c55ad2326515 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockPropertyValue.java @@ -16,10 +16,13 @@ */ package org.apache.nifi.util; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; +import org.apache.nifi.attribute.expression.language.Query; import org.apache.nifi.attribute.expression.language.StandardPropertyValue; +import org.apache.nifi.attribute.expression.language.Query.Range; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.ControllerService; @@ -219,4 +222,14 @@ public boolean isSet() { public String toString() { return getValue(); } + + @Override + public boolean isExpressionLanguagePresent() { + if (!expectExpressions) { + return false; + } + + final List elRanges = Query.extractExpressionRanges(rawValue); + return (elRanges != null && !elRanges.isEmpty()); + } } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java index 7c4ce77d32c8..0d755b0b4680 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/scheduling/ConnectableProcessContext.java @@ -166,6 +166,11 @@ public PropertyValue evaluateAttributeExpressions(FlowFile flowFile, Map + + diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java index 13b1d5da67b0..a5964c4bff59 100644 --- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java +++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.schemaregistry.services; +import java.io.IOException; import java.util.ArrayList; import java.util.HashMap; import java.util.List; @@ -34,11 +35,13 @@ import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.SimpleRecordSchema; 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.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; @Tags({"schema", "registry", "avro", "json", "csv"}) @CapabilityDescription("Provides a service for registering and accessing schemas. You can register a schema " @@ -54,39 +57,51 @@ public class AvroSchemaRegistry extends AbstractControllerService implements Sch private static final String LOGICAL_TYPE_TIMESTAMP_MILLIS = "timestamp-millis"; private static final String LOGICAL_TYPE_TIMESTAMP_MICROS = "timestamp-micros"; - public AvroSchemaRegistry() { this.schemaNameToSchemaMap = new HashMap<>(); } - @OnEnabled - public void enable(ConfigurationContext configuratiponContext) throws InitializationException { - this.schemaNameToSchemaMap.putAll(configuratiponContext.getProperties().entrySet().stream() - .filter(propEntry -> propEntry.getKey().isDynamic()) - .collect(Collectors.toMap(propEntry -> propEntry.getKey().getName(), propEntry -> propEntry.getValue()))); + @Override + public String retrieveSchemaText(final String schemaName) throws SchemaNotFoundException { + final String schemaText = schemaNameToSchemaMap.get(schemaName); + if (schemaText == null) { + throw new SchemaNotFoundException("Unable to find schema with name '" + schemaName + "'"); + } + + return schemaText; } @Override - public String retrieveSchemaText(String schemaName) { - if (!this.schemaNameToSchemaMap.containsKey(schemaName)) { - throw new IllegalArgumentException("Failed to find schema; Name: '" + schemaName + "."); - } else { - return this.schemaNameToSchemaMap.get(schemaName); - } + public RecordSchema retrieveSchema(final String schemaName) throws SchemaNotFoundException { + final String schemaText = retrieveSchemaText(schemaName); + final Schema schema = new Schema.Parser().parse(schemaText); + return createRecordSchema(schema, schemaText, schemaName); } @Override - public String retrieveSchemaText(String schemaName, Map attributes) { - throw new UnsupportedOperationException("This version of schema registry does not " - + "support this operation, since schemas are only identofied by name."); + public RecordSchema retrieveSchema(long schemaId, int version) throws IOException, SchemaNotFoundException { + throw new SchemaNotFoundException("This Schema Registry does not support schema lookup by identifier and version - only by name."); } @Override + public String retrieveSchemaText(long schemaId, int version) throws IOException, SchemaNotFoundException { + throw new SchemaNotFoundException("This Schema Registry does not support schema lookup by identifier and version - only by name."); + } + @OnDisabled public void close() throws Exception { - this.schemaNameToSchemaMap.clear(); + schemaNameToSchemaMap.clear(); } + + @OnEnabled + public void enable(final ConfigurationContext configurationContext) throws InitializationException { + this.schemaNameToSchemaMap.putAll(configurationContext.getProperties().entrySet().stream() + .filter(propEntry -> propEntry.getKey().isDynamic()) + .collect(Collectors.toMap(propEntry -> propEntry.getKey().getName(), propEntry -> propEntry.getValue()))); + } + + @Override protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) { return new PropertyDescriptor.Builder() @@ -99,20 +114,15 @@ protected PropertyDescriptor getSupportedDynamicPropertyDescriptor(final String } - @Override - public RecordSchema retrieveSchema(String schemaName) { - final String schemaText = this.retrieveSchemaText(schemaName); - final Schema schema = new Schema.Parser().parse(schemaText); - return createRecordSchema(schema); - } - /** * Converts an Avro Schema to a RecordSchema * * @param avroSchema the Avro Schema to convert + * @param text the textual representation of the schema + * @param schemaName the name of the schema * @return the Corresponding Record Schema */ - private RecordSchema createRecordSchema(final Schema avroSchema) { + private RecordSchema createRecordSchema(final Schema avroSchema, final String text, final String schemaName) { final List recordFields = new ArrayList<>(avroSchema.getFields().size()); for (final Field field : avroSchema.getFields()) { final String fieldName = field.name(); @@ -120,7 +130,7 @@ private RecordSchema createRecordSchema(final Schema avroSchema) { recordFields.add(new RecordField(fieldName, dataType)); } - final RecordSchema recordSchema = new SimpleRecordSchema(recordFields); + final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, text, "avro", SchemaIdentifier.ofName(schemaName)); return recordSchema; } @@ -178,12 +188,15 @@ private DataType determineDataType(final Schema avroSchema) { recordFields.add(new RecordField(fieldName, fieldType)); } - final RecordSchema recordSchema = new SimpleRecordSchema(recordFields); + final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), "avro", SchemaIdentifier.EMPTY); return RecordFieldType.RECORD.getRecordDataType(recordSchema); } case NULL: + return RecordFieldType.STRING.getDataType(); case MAP: - return RecordFieldType.RECORD.getDataType(); + final Schema valueSchema = avroSchema.getValueType(); + final DataType valueType = determineDataType(valueSchema); + return RecordFieldType.MAP.getMapDataType(valueType); case UNION: { final List nonNullSubSchemas = avroSchema.getTypes().stream() .filter(s -> s.getType() != Type.NULL) @@ -205,13 +218,4 @@ private DataType determineDataType(final Schema avroSchema) { return null; } - - /* - * For this implementation 'attributes' argument is ignored since the underlying storage mechanisms - * is based strictly on key/value pairs. In other implementation additional attributes may play a role (e.g., version id,) - */ - @Override - public RecordSchema retrieveSchema(String schemaName, Map attributes) { - return this.retrieveSchema(schemaName); - } } diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java index 929aab919066..a63097a4b13c 100644 --- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java +++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/test/java/org/apache/nifi/schemaregistry/services/TestAvroSchemaRegistry.java @@ -17,7 +17,6 @@ package org.apache.nifi.schemaregistry.services; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -27,9 +26,11 @@ import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; +import org.junit.Assert; import org.junit.Test; public class TestAvroSchemaRegistry { @@ -55,17 +56,17 @@ public void validateSchemaRegistrationFromrDynamicProperties() throws Exception properties.put(fooSchema, fooSchemaText); properties.put(barSchema, ""); when(configContext.getProperties()).thenReturn(properties); - SchemaRegistry delegate = new AvroSchemaRegistry(); - ((AvroSchemaRegistry) delegate).enable(configContext); + AvroSchemaRegistry delegate = new AvroSchemaRegistry(); + delegate.enable(configContext); String locatedSchemaText = delegate.retrieveSchemaText(schemaName); assertEquals(fooSchemaText, locatedSchemaText); try { - locatedSchemaText = delegate.retrieveSchemaText("barSchema"); - fail(); - } catch (Exception e) { - // ignore + delegate.retrieveSchemaText("barSchema"); + Assert.fail("Expected a SchemaNotFoundException to be thrown but it was not"); + } catch (final SchemaNotFoundException expected) { } + delegate.close(); } @@ -91,8 +92,8 @@ public void validateRecordSchemaRetrieval() throws Exception { properties.put(fooSchema, fooSchemaText); properties.put(barSchema, ""); when(configContext.getProperties()).thenReturn(properties); - SchemaRegistry delegate = new AvroSchemaRegistry(); - ((AvroSchemaRegistry) delegate).enable(configContext); + AvroSchemaRegistry delegate = new AvroSchemaRegistry(); + delegate.enable(configContext); RecordSchema locatedSchema = delegate.retrieveSchema(schemaName); List recordFields = locatedSchema.getFields(); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java similarity index 82% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java index 83a3d4b2798d..3de0a79307a8 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryFlowFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java @@ -16,8 +16,10 @@ */ package org.apache.nifi.processors.standard; +import java.io.BufferedInputStream; import java.io.Closeable; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.sql.Connection; import java.sql.DriverManager; @@ -69,10 +71,10 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.OutputStreamCallback; -import org.apache.nifi.queryflowfile.FlowFileTable; +import org.apache.nifi.queryrecord.FlowFileTable; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; -import org.apache.nifi.serialization.RowRecordReaderFactory; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.ResultSetRecordSet; import org.apache.nifi.util.StopWatch; @@ -96,12 +98,12 @@ @DynamicProperty(name = "The name of the relationship to route data to", value="A SQL SELECT statement that is used to determine what data should be routed to this " + "relationship.", supportsExpressionLanguage=true, description="Each user-defined property specifies a SQL SELECT statement to run over the data, with the data " + "that is selected being routed to the relationship whose name is the property name") -public class QueryFlowFile extends AbstractProcessor { +public class QueryRecord extends AbstractProcessor { static final PropertyDescriptor RECORD_READER_FACTORY = new PropertyDescriptor.Builder() .name("record-reader") .displayName("Record Reader") .description("Specifies the Controller Service to use for parsing incoming data and determining the data's schema") - .identifiesControllerService(RowRecordReaderFactory.class) + .identifiesControllerService(RecordReaderFactory.class) .required(true) .build(); static final PropertyDescriptor RECORD_WRITER_FACTORY = new PropertyDescriptor.Builder() @@ -240,14 +242,21 @@ public void onTrigger(final ProcessContext context, final ProcessSession session final RecordSetWriterFactory resultSetWriterFactory = context.getProperty(RECORD_WRITER_FACTORY) .asControllerService(RecordSetWriterFactory.class); - final RowRecordReaderFactory recordParserFactory = context.getProperty(RECORD_READER_FACTORY) - .asControllerService(RowRecordReaderFactory.class); + final RecordReaderFactory recordParserFactory = context.getProperty(RECORD_READER_FACTORY) + .asControllerService(RecordReaderFactory.class); - final RecordSetWriter resultSetWriter = resultSetWriterFactory.createWriter(getLogger()); final Map transformedFlowFiles = new HashMap<>(); final Set createdFlowFiles = new HashSet<>(); + int recordsRead = 0; + try { + final RecordSetWriter resultSetWriter; + try (final InputStream rawIn = session.read(original); + final InputStream in = new BufferedInputStream(rawIn)) { + resultSetWriter = resultSetWriterFactory.createWriter(getLogger(), original, in); + } + for (final PropertyDescriptor descriptor : context.getProperties().keySet()) { if (!descriptor.isDynamic()) { continue; @@ -260,50 +269,55 @@ public void onTrigger(final ProcessContext context, final ProcessSession session // callback for the original FlowFile. FlowFile transformed = session.create(original); - // Ensure that we have the FlowFile in the map in case we throw any Exception - createdFlowFiles.add(transformed); - - final String sql = context.getProperty(descriptor).evaluateAttributeExpressions(original).getValue(); - final AtomicReference writeResultRef = new AtomicReference<>(); - final QueryResult queryResult; - if (context.getProperty(CACHE_SCHEMA).asBoolean()) { - queryResult = queryWithCache(session, original, sql, context, recordParserFactory); - } else { - queryResult = query(session, original, sql, context, recordParserFactory); - } - try { - final ResultSet rs = queryResult.getResultSet(); - transformed = session.write(transformed, new OutputStreamCallback() { - @Override - public void process(final OutputStream out) throws IOException { - try { - final ResultSetRecordSet recordSet = new ResultSetRecordSet(rs); - writeResultRef.set(resultSetWriter.write(recordSet, out)); - } catch (final Exception e) { - throw new IOException(e); + final String sql = context.getProperty(descriptor).evaluateAttributeExpressions(original).getValue(); + final AtomicReference writeResultRef = new AtomicReference<>(); + final QueryResult queryResult; + if (context.getProperty(CACHE_SCHEMA).asBoolean()) { + queryResult = queryWithCache(session, original, sql, context, recordParserFactory); + } else { + queryResult = query(session, original, sql, context, recordParserFactory); + } + + try { + final ResultSet rs = queryResult.getResultSet(); + transformed = session.write(transformed, new OutputStreamCallback() { + @Override + public void process(final OutputStream out) throws IOException { + try { + final ResultSetRecordSet recordSet = new ResultSetRecordSet(rs); + writeResultRef.set(resultSetWriter.write(recordSet, out)); + } catch (final Exception e) { + throw new IOException(e); + } } + }); + } finally { + closeQuietly(queryResult); + } + + recordsRead = Math.max(recordsRead, queryResult.getRecordsRead()); + final WriteResult result = writeResultRef.get(); + if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) { + session.remove(transformed); + transformedFlowFiles.remove(transformed); + getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original}); + } else { + final Map attributesToAdd = new HashMap<>(); + if (result.getAttributes() != null) { + attributesToAdd.putAll(result.getAttributes()); } - }); - } finally { - closeQuietly(queryResult); - } - final WriteResult result = writeResultRef.get(); - if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) { - session.remove(transformed); - transformedFlowFiles.remove(transformed); - getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original}); - } else { - final Map attributesToAdd = new HashMap<>(); - if (result.getAttributes() != null) { - attributesToAdd.putAll(result.getAttributes()); - } + attributesToAdd.put(CoreAttributes.MIME_TYPE.key(), resultSetWriter.getMimeType()); + attributesToAdd.put("record.count", String.valueOf(result.getRecordCount())); + transformed = session.putAllAttributes(transformed, attributesToAdd); + transformedFlowFiles.put(transformed, relationship); - attributesToAdd.put(CoreAttributes.MIME_TYPE.key(), resultSetWriter.getMimeType()); - attributesToAdd.put("record.count", String.valueOf(result.getRecordCount())); - transformed = session.putAllAttributes(transformed, attributesToAdd); - transformedFlowFiles.put(transformed, relationship); + session.adjustCounter("Records Written", result.getRecordCount(), false); + } + } finally { + // Ensure that we have the FlowFile in the set in case we throw any Exception + createdFlowFiles.add(transformed); } } @@ -320,21 +334,23 @@ public void process(final OutputStream out) throws IOException { } } - getLogger().info("Successfully transformed {} in {} millis", new Object[] {original, elapsedMillis}); + getLogger().info("Successfully queried {} in {} millis", new Object[] {original, elapsedMillis}); session.transfer(original, REL_ORIGINAL); - } catch (ProcessException e) { - getLogger().error("Unable to transform {} due to {}", new Object[] {original, e}); + } catch (final SQLException e) { + getLogger().error("Unable to query {} due to {}", new Object[] {original, e.getCause() == null ? e : e.getCause()}); session.remove(createdFlowFiles); session.transfer(original, REL_FAILURE); - } catch (final SQLException e) { - getLogger().error("Unable to transform {} due to {}", new Object[] {original, e.getCause() == null ? e : e.getCause()}); + } catch (final Exception e) { + getLogger().error("Unable to query {} due to {}", new Object[] {original, e}); session.remove(createdFlowFiles); session.transfer(original, REL_FAILURE); } + + session.adjustCounter("Records Read", recordsRead, false); } private synchronized CachedStatement getStatement(final String sql, final Supplier connectionSupplier, final ProcessSession session, - final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException { + final FlowFile flowFile, final RecordReaderFactory recordReaderFactory) throws SQLException { final BlockingQueue statementQueue = statementQueues.get(sql); if (statementQueue == null) { @@ -350,7 +366,7 @@ private synchronized CachedStatement getStatement(final String sql, final Suppli } private CachedStatement buildCachedStatement(final String sql, final Supplier connectionSupplier, final ProcessSession session, - final FlowFile flowFile, final RowRecordReaderFactory recordReaderFactory) throws SQLException { + final FlowFile flowFile, final RecordReaderFactory recordReaderFactory) throws SQLException { final CalciteConnection connection = connectionSupplier.get(); final SchemaPlus rootSchema = connection.getRootSchema(); @@ -391,7 +407,7 @@ public synchronized void setupQueues(final ProcessContext context) { } protected QueryResult queryWithCache(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context, - final RowRecordReaderFactory recordParserFactory) throws SQLException { + final RecordReaderFactory recordParserFactory) throws SQLException { final Supplier connectionSupplier = () -> { final Properties properties = new Properties(); @@ -430,11 +446,17 @@ public void close() throws IOException { public ResultSet getResultSet() { return rs; } + + @Override + public int getRecordsRead() { + return table.getRecordsRead(); + } + }; } protected QueryResult query(final ProcessSession session, final FlowFile flowFile, final String sql, final ProcessContext context, - final RowRecordReaderFactory recordParserFactory) throws SQLException { + final RecordReaderFactory recordParserFactory) throws SQLException { final Properties properties = new Properties(); properties.put(CalciteConnectionProperty.LEX.camelName(), Lex.JAVA.name()); @@ -467,6 +489,11 @@ public void close() throws IOException { public ResultSet getResultSet() { return rs; } + + @Override + public int getRecordsRead() { + return flowFileTable.getRecordsRead(); + } }; } catch (final Exception e) { closeQuietly(resultSet, statement, connection); @@ -526,6 +553,8 @@ public ValidationResult validate(final String subject, final String input, final private static interface QueryResult extends Closeable { ResultSet getResultSet(); + + int getRecordsRead(); } private static class CachedStatement { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java similarity index 82% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java index 7daa002ceb98..e7b2c269db48 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileEnumerator.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileEnumerator.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.nifi.queryflowfile; +package org.apache.nifi.queryrecord; -import java.io.IOException; import java.io.InputStream; import org.apache.calcite.linq4j.Enumerator; @@ -25,23 +24,23 @@ import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.record.Record; public class FlowFileEnumerator implements Enumerator { private final ProcessSession session; private final FlowFile flowFile; private final ComponentLog logger; - private final RowRecordReaderFactory recordParserFactory; + private final RecordReaderFactory recordParserFactory; private final int[] fields; private InputStream rawIn; private Object currentRow; private RecordReader recordParser; + private int recordsRead = 0; - public FlowFileEnumerator(final ProcessSession session, final FlowFile flowFile, final ComponentLog logger, final RowRecordReaderFactory parserFactory, final int[] fields) { + public FlowFileEnumerator(final ProcessSession session, final FlowFile flowFile, final ComponentLog logger, final RecordReaderFactory parserFactory, final int[] fields) { this.session = session; this.flowFile = flowFile; this.recordParserFactory = parserFactory; @@ -62,12 +61,8 @@ public boolean moveNext() { try { currentRow = filterColumns(recordParser.nextRecord()); break; - } catch (final IOException e) { - logger.error("Failed to read next record in stream for " + flowFile + ". Assuming end of stream.", e); - currentRow = null; - break; - } catch (final MalformedRecordException mre) { - logger.error("Failed to parse record in stream for " + flowFile + ". Will skip record and continue reading", mre); + } catch (final Exception e) { + throw new ProcessException("Failed to read next record in stream for " + flowFile, e); } } @@ -75,8 +70,24 @@ public boolean moveNext() { // If we are out of data, close the InputStream. We do this because // Calcite does not necessarily call our close() method. close(); + try { + onFinish(); + } catch (final Exception e) { + logger.error("Failed to perform tasks when enumerator was finished", e); + } + + return false; } - return (currentRow != null); + + recordsRead++; + return true; + } + + protected int getRecordsRead() { + return recordsRead; + } + + protected void onFinish() { } private Object filterColumns(final Record record) { @@ -122,7 +133,7 @@ public void reset() { try { recordParser = recordParserFactory.createRecordReader(flowFile, rawIn, logger); - } catch (final MalformedRecordException | IOException e) { + } catch (final Exception e) { throw new ProcessException("Failed to reset stream", e); } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java similarity index 98% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java index c5179c9aab6e..a44a5d8cf8b2 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileProjectTableScanRule.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileProjectTableScanRule.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.queryflowfile; +package org.apache.nifi.queryrecord; import java.util.List; diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java similarity index 88% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java index 27f0c42f8afb..bd15dc2cdbb5 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTable.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java @@ -14,12 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.queryflowfile; +package org.apache.nifi.queryrecord; -import java.io.IOException; import java.io.InputStream; import java.lang.reflect.Type; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; import org.apache.calcite.adapter.java.JavaTypeFactory; @@ -44,9 +44,8 @@ import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.exception.ProcessException; -import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordSchema; @@ -54,7 +53,7 @@ public class FlowFileTable extends AbstractTable implements QueryableTable, TranslatableTable { - private final RowRecordReaderFactory recordParserFactory; + private final RecordReaderFactory recordParserFactory; private final ComponentLog logger; private RecordSchema recordSchema; @@ -62,11 +61,12 @@ public class FlowFileTable extends AbstractTable implements QueryableTable private volatile ProcessSession session; private volatile FlowFile flowFile; + private volatile int maxRecordsRead; /** * Creates a FlowFile table. */ - public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RowRecordReaderFactory recordParserFactory, final ComponentLog logger) { + public FlowFileTable(final ProcessSession session, final FlowFile flowFile, final RecordReaderFactory recordParserFactory, final ComponentLog logger) { this.session = session; this.flowFile = flowFile; this.recordParserFactory = recordParserFactory; @@ -76,6 +76,7 @@ public FlowFileTable(final ProcessSession session, final FlowFile flowFile, fina public void setFlowFile(final ProcessSession session, final FlowFile flowFile) { this.session = session; this.flowFile = flowFile; + this.maxRecordsRead = 0; } @@ -95,11 +96,23 @@ public Enumerable project(final int[] fields) { @Override @SuppressWarnings({"unchecked", "rawtypes"}) public Enumerator enumerator() { - return new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields); + return new FlowFileEnumerator(session, flowFile, logger, recordParserFactory, fields) { + @Override + protected void onFinish() { + final int recordCount = getRecordsRead(); + if (recordCount > maxRecordsRead) { + maxRecordsRead = recordCount; + } + } + }; } }; } + public int getRecordsRead() { + return maxRecordsRead; + } + @Override @SuppressWarnings("rawtypes") public Expression getExpression(final SchemaPlus schema, final String tableName, final Class clazz) { @@ -138,7 +151,7 @@ public RelDataType getRowType(final RelDataTypeFactory typeFactory) { try (final InputStream in = session.read(flowFile)) { final RecordReader recordParser = recordParserFactory.createRecordReader(flowFile, in, logger); schema = recordParser.getSchema(); - } catch (final MalformedRecordException | IOException e) { + } catch (final Exception e) { throw new ProcessException("Failed to determine schema of data records for " + flowFile, e); } @@ -191,6 +204,8 @@ private RelDataType getRelDataType(final DataType fieldType, final JavaTypeFacto return typeFactory.createJavaType(Object[].class); case RECORD: return typeFactory.createJavaType(Object.class); + case MAP: + return typeFactory.createJavaType(HashMap.class); } throw new IllegalArgumentException("Unknown Record Field Type: " + fieldType); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java similarity index 98% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java index ad3a1c36eeef..afca2027f69e 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryflowfile/FlowFileTableScan.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTableScan.java @@ -14,7 +14,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.nifi.queryflowfile; +package org.apache.nifi.queryrecord; import java.util.List; 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 3891ee620c46..d85e66312a22 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 @@ -76,7 +76,7 @@ org.apache.nifi.processors.standard.PutSyslog org.apache.nifi.processors.standard.PutTCP org.apache.nifi.processors.standard.PutUDP org.apache.nifi.processors.standard.QueryDatabaseTable -org.apache.nifi.processors.standard.QueryFlowFile +org.apache.nifi.processors.standard.QueryRecord org.apache.nifi.processors.standard.ReplaceText org.apache.nifi.processors.standard.RouteText org.apache.nifi.processors.standard.ReplaceTextWithMapping diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html similarity index 94% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html index 0dffc0dc0895..93bbe2a76b84 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryFlowFile/additionalDetails.html +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/resources/docs/org.apache.nifi.processors.standard.QueryRecord/additionalDetails.html @@ -16,14 +16,14 @@ --> - QueryFlowFile + QueryRecord

- QueryFlowFile provides users a tremendous amount of power by leveraging an extremely well-known + QueryRecord provides users a tremendous amount of power by leveraging an extremely well-known syntax (SQL) to route, filter, transform, and query data as it traverses the system. In order to provide the Processor with the maximum amount of flexibility, it is configured with a Controller Service that is responsible for reading and parsing the incoming FlowFiles and a Controller Service diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java similarity index 91% rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java index 8e1c7edb09a1..65002c68c468 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryFlowFile.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java @@ -17,6 +17,7 @@ package org.apache.nifi.processors.standard; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.sql.SQLException; import java.util.ArrayList; @@ -24,6 +25,7 @@ import java.util.List; import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processors.standard.util.record.MockRecordParser; import org.apache.nifi.processors.standard.util.record.MockRecordWriter; @@ -40,7 +42,7 @@ import org.junit.Assert; import org.junit.Test; -public class TestQueryFlowFile { +public class TestQueryRecord { static { System.setProperty("org.slf4j.simpleLogger.defaultLogLevel", "info"); @@ -60,15 +62,15 @@ public void testSimple() throws InitializationException, IOException, SQLExcepti final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\""); - final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class); + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); runner.addControllerService("parser", parser); runner.enableControllerService(parser); runner.addControllerService("writer", writer); runner.enableControllerService(writer); runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''"); - runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser"); - runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); final int numIterations = 1; for (int i = 0; i < numIterations; i++) { @@ -93,15 +95,15 @@ public void testParseFailure() throws InitializationException, IOException, SQLE final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\""); - final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class); + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); runner.addControllerService("parser", parser); runner.enableControllerService(parser); runner.addControllerService("writer", writer); runner.enableControllerService(writer); runner.setProperty(REL_NAME, "select name, age from FLOWFILE WHERE name <> ''"); - runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser"); - runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); final int numIterations = 1; for (int i = 0; i < numIterations; i++) { @@ -133,15 +135,15 @@ public void testTransformCalc() throws InitializationException, IOException, SQL final MockRecordWriter writer = new MockRecordWriter("\"NAME\",\"POINTS\""); - final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class); + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); runner.addControllerService("parser", parser); runner.enableControllerService(parser); runner.addControllerService("writer", writer); runner.enableControllerService(writer); runner.setProperty(REL_NAME, "select ID, AMOUNT1+AMOUNT2+AMOUNT3 as TOTAL from FLOWFILE where ID=100"); - runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser"); - runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); runner.enqueue(new byte[0]); runner.run(); @@ -164,15 +166,15 @@ public void testAggregateFunction() throws InitializationException, IOException final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\""); - final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class); + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); runner.addControllerService("parser", parser); runner.enableControllerService(parser); runner.addControllerService("writer", writer); runner.enableControllerService(writer); runner.setProperty(REL_NAME, "select name, sum(points) as points from FLOWFILE GROUP BY name"); - runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser"); - runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); runner.enqueue(""); runner.run(); @@ -199,15 +201,15 @@ public void testColumnNames() throws InitializationException, IOException { colNames.add("FAV_GREETING"); final ResultSetValidatingRecordWriter writer = new ResultSetValidatingRecordWriter(colNames); - final TestRunner runner = TestRunners.newTestRunner(QueryFlowFile.class); + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); runner.addControllerService("parser", parser); runner.enableControllerService(parser); runner.addControllerService("writer", writer); runner.enableControllerService(writer); runner.setProperty(REL_NAME, "select *, greeting AS FAV_GREETING from FLOWFILE"); - runner.setProperty(QueryFlowFile.RECORD_READER_FACTORY, "parser"); - runner.setProperty(QueryFlowFile.RECORD_WRITER_FACTORY, "writer"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); runner.enqueue(""); runner.run(); @@ -224,7 +226,7 @@ public ResultSetValidatingRecordWriter(final List colNames) { } @Override - public RecordSetWriter createWriter(ComponentLog logger) { + public RecordSetWriter createWriter(ComponentLog logger, FlowFile flowFile, InputStream in) { return new RecordSetWriter() { @Override public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java index 1a39b826a4ff..e78fddd53667 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java @@ -30,7 +30,7 @@ import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.Record; @@ -38,7 +38,7 @@ import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; -public class MockRecordParser extends AbstractControllerService implements RowRecordReaderFactory { +public class MockRecordParser extends AbstractControllerService implements RecordReaderFactory { private final List records = new ArrayList<>(); private final List fields = new ArrayList<>(); private final int failAfterN; @@ -99,9 +99,4 @@ public RecordSchema getSchema() { } }; } - - @Override - public RecordSchema getSchema(FlowFile flowFile) throws MalformedRecordException, IOException { - return null; - } } \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java index 0a57b29f1472..df3e0f18c84b 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java @@ -18,10 +18,12 @@ package org.apache.nifi.processors.standard.util.record; import java.io.IOException; +import java.io.InputStream; import java.io.OutputStream; import java.util.Collections; import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; @@ -49,7 +51,7 @@ public MockRecordWriter(final String header, final boolean quoteValues, final in } @Override - public RecordSetWriter createWriter(final ComponentLog logger) { + public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) { return new RecordSetWriter() { @Override public WriteResult write(final RecordSet rs, final OutputStream out) throws IOException { diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java new file mode 100644 index 000000000000..9a064ffd827d --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaNotFoundException.java @@ -0,0 +1,32 @@ +/* + * 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.schema.access; + +public class SchemaNotFoundException extends Exception { + public SchemaNotFoundException(final String message) { + super(message); + } + + public SchemaNotFoundException(final String message, final Throwable cause) { + super(cause); + } + + public SchemaNotFoundException(final Throwable cause) { + super(cause); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java similarity index 84% rename from nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java rename to nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java index fbd8a215033b..7d7268e2e4c7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RowRecordReaderFactory.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordReaderFactory.java @@ -23,16 +23,15 @@ import org.apache.nifi.controller.ControllerService; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.schema.access.SchemaNotFoundException; /** *

* A Controller Service that is responsible for creating a {@link RecordReader}. *

*/ -public interface RowRecordReaderFactory extends ControllerService { +public interface RecordReaderFactory extends ControllerService { - RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException; + RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException; - RecordSchema getSchema(FlowFile flowFile) throws MalformedRecordException, IOException; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java index 2286f3f9f837..e23ad203391f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/RecordSetWriterFactory.java @@ -17,14 +17,47 @@ package org.apache.nifi.serialization; +import java.io.IOException; +import java.io.InputStream; + import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; /** *

- * A Controller Service that is responsible for creating a {@link RecordSetWriter}. + * A Controller Service that is responsible for creating a {@link RecordSetWriter}. The writer is created + * based on a FlowFile and an InputStream for that FlowFile, but it is important to note that this the FlowFile passed + * to the {@link #createWriter(ComponentLog, FlowFile, InputStream)} may not be the FlowFile that the Writer will writer to. + * Rather, it is the FlowFile and InputStream from which the Writer's Schema should be determined. This is done because most + * Processors that make use of Record Writers also make use of Record Readers and the schema for the output is often determined + * by either reading the schema from the content of the input FlowFile or from referencing attributes of the + * input FlowFile. + *

+ * + *

+ * PLEASE NOTE: This interface is still considered 'unstable' and may change in a non-backward-compatible + * manner between minor or incremental releases of NiFi. *

*/ public interface RecordSetWriterFactory extends ControllerService { - RecordSetWriter createWriter(ComponentLog logger); + + /** + *

+ * Creates a new RecordSetWriter that is capable of writing record contents to an OutputStream. Note that the + * FlowFile and InputStream that are given may well be different than the FlowFile that the writer is intended + * to write to. The given FlowFile and InputStream are intended to be used for determining the schema that should + * be used when writing records. + *

+ * + * @param logger the logger to use when logging information. This is passed in, rather than using the logger of the Controller Service + * because it allows messages to be logged for the component that is calling this Controller Service. + * @param schemaFlowFile the FlowFile from which the schema should be determined. + * @param schemaFlowFileContent the contents of the FlowFile from which to determine the schema + * @return a RecordSetWriter that can write record sets to an OutputStream + * @throws SchemaNotFoundException if unable to find the schema + * @throws IOException if unable to read from the given InputStream + */ + RecordSetWriter createWriter(ComponentLog logger, FlowFile schemaFlowFile, InputStream schemaFlowFileContent) throws SchemaNotFoundException, IOException; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java index 246e0af956dd..576fda92b287 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java @@ -29,12 +29,33 @@ import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; public class SimpleRecordSchema implements RecordSchema { private final List fields; private final Map fieldIndices; + private final boolean textAvailable; + private final String text; + private final String schemaFormat; + private final SchemaIdentifier schemaIdentifier; public SimpleRecordSchema(final List fields) { + this(fields, createText(fields), null, false, SchemaIdentifier.EMPTY); + } + + public SimpleRecordSchema(final List fields, final SchemaIdentifier id) { + this(fields, createText(fields), null, false, id); + } + + public SimpleRecordSchema(final List fields, final String text, final String schemaFormat, final SchemaIdentifier id) { + this(fields, text, schemaFormat, true, id); + } + + public SimpleRecordSchema(final List fields, final String text, final String schemaFormat, final boolean textAvailable, final SchemaIdentifier id) { + this.text = text; + this.schemaFormat = schemaFormat; + this.schemaIdentifier = id; + this.textAvailable = textAvailable; this.fields = Collections.unmodifiableList(new ArrayList<>(fields)); this.fieldIndices = new HashMap<>(fields.size()); @@ -44,6 +65,21 @@ public SimpleRecordSchema(final List fields) { } } + @Override + public Optional getSchemaText() { + if (textAvailable) { + return Optional.ofNullable(text); + } else { + return Optional.empty(); + } + } + + + @Override + public Optional getSchemaFormat() { + return Optional.ofNullable(schemaFormat); + } + @Override public List getFields() { return fields; @@ -103,8 +139,7 @@ public int hashCode() { return 143 + 3 * fields.hashCode(); } - @Override - public String toString() { + private static String createText(final List fields) { final StringBuilder sb = new StringBuilder("["); for (int i = 0; i < fields.size(); i++) { @@ -123,4 +158,14 @@ public String toString() { sb.append("]"); return sb.toString(); } + + @Override + public String toString() { + return text; + } + + @Override + public SchemaIdentifier getIdentifier() { + return schemaIdentifier; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java index b72c107e9b15..6ed4bd6f88f5 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/DataType.java @@ -17,6 +17,8 @@ package org.apache.nifi.serialization.record; +import java.util.Objects; + public class DataType { private final RecordFieldType fieldType; private final String format; @@ -36,7 +38,7 @@ public RecordFieldType getFieldType() { @Override public int hashCode() { - return 31 + 41 * fieldType.hashCode() + 41 * (format == null ? 0 : format.hashCode()); + return 31 + 41 * getFieldType().hashCode() + 41 * (getFormat() == null ? 0 : getFormat().hashCode()); } @Override @@ -52,15 +54,15 @@ public boolean equals(final Object obj) { } final DataType other = (DataType) obj; - return fieldType.equals(other.fieldType) && ((format == null && other.format == null) || (format != null && format.equals(other.format))); + return getFieldType().equals(other.getFieldType()) && Objects.equals(getFormat(), other.getFormat()); } @Override public String toString() { - if (format == null) { - return fieldType.toString(); + if (getFormat() == null) { + return getFieldType().toString(); } else { - return fieldType.toString() + ":" + format; + return getFieldType().toString() + ":" + getFormat(); } } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java index 0bbb534e6ffe..750bd98b29d3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java @@ -85,42 +85,42 @@ private String convertToString(final Object value, final String format) { @Override public Long getAsLong(final String fieldName) { - return DataTypeUtils.toLong(getValue(fieldName)); + return DataTypeUtils.toLong(getValue(fieldName), fieldName); } @Override public Integer getAsInt(final String fieldName) { - return DataTypeUtils.toInteger(getValue(fieldName)); + return DataTypeUtils.toInteger(getValue(fieldName), fieldName); } @Override public Double getAsDouble(final String fieldName) { - return DataTypeUtils.toDouble(getValue(fieldName)); + return DataTypeUtils.toDouble(getValue(fieldName), fieldName); } @Override public Float getAsFloat(final String fieldName) { - return DataTypeUtils.toFloat(getValue(fieldName)); + return DataTypeUtils.toFloat(getValue(fieldName), fieldName); } @Override public Record getAsRecord(String fieldName, final RecordSchema schema) { - return DataTypeUtils.toRecord(getValue(fieldName), schema); + return DataTypeUtils.toRecord(getValue(fieldName), schema, fieldName); } @Override public Boolean getAsBoolean(final String fieldName) { - return DataTypeUtils.toBoolean(getValue(fieldName)); + return DataTypeUtils.toBoolean(getValue(fieldName), fieldName); } @Override public Date getAsDate(final String fieldName, final String format) { - return DataTypeUtils.toDate(getValue(fieldName), format); + return DataTypeUtils.toDate(getValue(fieldName), format, fieldName); } @Override public Object[] getAsArray(final String fieldName) { - return DataTypeUtils.toArray(getValue(fieldName)); + return DataTypeUtils.toArray(getValue(fieldName), fieldName); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java index cc83a4164e3d..785b8d2b030e 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordFieldType.java @@ -25,6 +25,7 @@ import org.apache.nifi.serialization.record.type.ArrayDataType; import org.apache.nifi.serialization.record.type.ChoiceDataType; +import org.apache.nifi.serialization.record.type.MapDataType; import org.apache.nifi.serialization.record.type.RecordDataType; public enum RecordFieldType { @@ -149,7 +150,7 @@ public enum RecordFieldType { /** *

- * An array field type. Records should be updated using an {@code Object[]} value for this field. Note that we are explicitly indicating that + * An array field type. Fields of this type use a {@code Object[]} value. Note that we are explicitly indicating that * Object[] should be used here and not primitive array types. For instance, setting a value of {@code int[]} is not allowed. The DataType for * this field should be created using the {@link #getArrayDataType(DataType)} method: *

@@ -173,7 +174,34 @@ public enum RecordFieldType { * * */ - ARRAY("array", null, new ArrayDataType(null)); + ARRAY("array", null, new ArrayDataType(null)), + + /** + *

+ * A record field type. Fields of this type use a {@code Map} value. A Map DataType should be + * created by providing the {@link DataType} for the values: + *

+ * + * + * final DataType recordType = RecordFieldType.MAP.getRecordDataType( RecordFieldType.STRING.getDataType() ); + * + * + *

+ * A field of type MAP should always have a {@link MapDataType}, so the following idiom is acceptable for use: + *

+ * + * + *
+     * final DataType dataType = ...;
+     * if (dataType.getFieldType() == RecordFieldType.MAP) {
+     *     final MapDataType mapDataType = (MapDataType) dataType;
+     *     final DataType valueType = mapDataType.getValueType();
+     *     ...
+     * }
+     * 
+ *
+ */ + MAP("map", null, new MapDataType(null)); private static final Map SIMPLE_NAME_MAP = new HashMap(); @@ -235,11 +263,11 @@ public DataType getRecordDataType(final RecordSchema childSchema) { } /** - * Returns a Data Type that represents a "RECORD" or "ARRAY" type with the given schema. + * Returns a Data Type that represents an "ARRAY" type with the given element type. * * @param elementType the type of the arrays in the element - * @return a DataType that represents a Record or Array with the given schema, or null if this RecordFieldType - * is not the RECORD or ARRAY type. + * @return a DataType that represents an Array with the given element type, or null if this RecordFieldType + * is not the ARRAY type. */ public DataType getArrayDataType(final DataType elementType) { if (this != ARRAY) { @@ -287,6 +315,21 @@ public DataType getChoiceDataType(final DataType... possibleChildTypes) { return new ChoiceDataType(list); } + /** + * Returns a Data Type that represents a "MAP" type with the given value type. + * + * @param valueDataType the type of the values in the map + * @return a DataType that represents a Map with the given value type, or null if this RecordFieldType + * is not the MAP type. + */ + public DataType getMapDataType(final DataType valueDataType) { + if (this != MAP) { + return null; + } + + return new MapDataType(valueDataType); + } + public static RecordFieldType of(final String typeString) { return SIMPLE_NAME_MAP.get(typeString); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java index 115fb5165db8..8215166e2fba 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java @@ -55,4 +55,19 @@ public interface RecordSchema { * null if the schema does not contain a field with the given name */ Optional getDataType(String fieldName); + + /** + * @return the textual representation of the schema, if one is available + */ + Optional getSchemaText(); + + /** + * @return the format of the schema text, if schema text is present + */ + Optional getSchemaFormat(); + + /** + * @return the SchemaIdentifier, which provides various attributes for identifying a schema + */ + SchemaIdentifier getIdentifier(); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java index be064ab11480..b6daab76c628 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/ResultSetRecordSet.java @@ -157,6 +157,20 @@ private static DataType getDataType(final int sqlType, final ResultSet rs, final case Types.LONGVARBINARY: case Types.VARBINARY: return RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()); + case Types.OTHER: + // If we have no records to inspect, we can't really know its schema so we simply use the default data type. + if (rs.isAfterLast()) { + return RecordFieldType.RECORD.getDataType(); + } + + final Object obj = rs.getObject(columnIndex); + if (obj == null || !(obj instanceof Record)) { + return RecordFieldType.RECORD.getDataType(); + } + + final Record record = (Record) obj; + final RecordSchema recordSchema = record.getSchema(); + return RecordFieldType.RECORD.getRecordDataType(recordSchema); default: return getFieldType(sqlType).getDataType(); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java new file mode 100644 index 000000000000..b7119525ec29 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/SchemaIdentifier.java @@ -0,0 +1,96 @@ +/* + * 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.Optional; +import java.util.OptionalInt; +import java.util.OptionalLong; + +public interface SchemaIdentifier { + + /** + * @return the name of the schema, if one has been defined. + */ + Optional getName(); + + /** + * @return the identifier of the schema, if one has been defined. + */ + OptionalLong getIdentifier(); + + /** + * @return the version of the schema, if one has been defined. + */ + OptionalInt getVersion(); + + + public static SchemaIdentifier EMPTY = new SchemaIdentifier() { + @Override + public Optional getName() { + return Optional.empty(); + } + + @Override + public OptionalLong getIdentifier() { + return OptionalLong.empty(); + } + + @Override + public OptionalInt getVersion() { + return OptionalInt.empty(); + } + }; + + public static SchemaIdentifier ofName(final String name) { + return new SchemaIdentifier() { + @Override + public Optional getName() { + return Optional.ofNullable(name); + } + + @Override + public OptionalLong getIdentifier() { + return OptionalLong.empty(); + } + + @Override + public OptionalInt getVersion() { + return OptionalInt.empty(); + } + }; + } + + public static SchemaIdentifier of(final String name, final long identifier, final int version) { + return new SchemaIdentifier() { + @Override + public Optional getName() { + return Optional.ofNullable(name); + } + + @Override + public OptionalLong getIdentifier() { + return OptionalLong.of(identifier); + } + + @Override + public OptionalInt getVersion() { + return OptionalInt.of(version); + } + }; + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java index f507f230cb3d..0c212396da2d 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ArrayDataType.java @@ -52,7 +52,7 @@ public boolean equals(final Object obj) { if (obj == null) { return false; } - if (!(obj instanceof RecordDataType)) { + if (!(obj instanceof ArrayDataType)) { return false; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java index b74cdccf32df..038b147d2c76 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/ChoiceDataType.java @@ -53,7 +53,7 @@ public boolean equals(final Object obj) { if (obj == null) { return false; } - if (!(obj instanceof RecordDataType)) { + if (!(obj instanceof ChoiceDataType)) { return false; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java new file mode 100644 index 000000000000..a85fb5e8de0a --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/MapDataType.java @@ -0,0 +1,67 @@ +/* + * 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.type; + +import java.util.Objects; + +import org.apache.nifi.serialization.record.DataType; +import org.apache.nifi.serialization.record.RecordFieldType; + +public class MapDataType extends DataType { + private final DataType valueType; + + public MapDataType(final DataType elementType) { + super(RecordFieldType.MAP, null); + this.valueType = elementType; + } + + public DataType getValueType() { + return valueType; + } + + @Override + public RecordFieldType getFieldType() { + return RecordFieldType.MAP; + } + + @Override + public int hashCode() { + return 31 + 41 * getFieldType().hashCode() + 41 * (valueType == null ? 0 : valueType.hashCode()); + } + + @Override + public boolean equals(final Object obj) { + if (obj == this) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof MapDataType)) { + return false; + } + + final MapDataType other = (MapDataType) obj; + return getValueType().equals(other.getValueType()) && Objects.equals(valueType, other.valueType); + } + + @Override + public String toString() { + return "MAP[" + valueType + "]"; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java index f24d0367f45f..006d34c737ed 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/type/RecordDataType.java @@ -60,4 +60,9 @@ public boolean equals(final Object obj) { final RecordDataType other = (RecordDataType) obj; return getFieldType().equals(other.getFieldType()) && Objects.equals(childSchema, other.childSchema); } + + @Override + public String toString() { + return RecordFieldType.RECORD.toString(); + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index 1cdefb85b4c9..798946ca898d 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -42,44 +42,46 @@ public class DataTypeUtils { private static final TimeZone gmt = TimeZone.getTimeZone("gmt"); - public static Object convertType(final Object value, final DataType dataType) { - return convertType(value, dataType, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); + public static Object convertType(final Object value, final DataType dataType, final String fieldName) { + return convertType(value, dataType, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat(), fieldName); } - public static Object convertType(final Object value, final DataType dataType, final String dateFormat, final String timeFormat, final String timestampFormat) { + public static Object convertType(final Object value, final DataType dataType, final String dateFormat, final String timeFormat, final String timestampFormat, final String fieldName) { switch (dataType.getFieldType()) { case BIGINT: - return toBigInt(value); + return toBigInt(value, fieldName); case BOOLEAN: - return toBoolean(value); + return toBoolean(value, fieldName); case BYTE: - return toByte(value); + return toByte(value, fieldName); case CHAR: - return toCharacter(value); + return toCharacter(value, fieldName); case DATE: - return toDate(value, dateFormat); + return toDate(value, dateFormat, fieldName); case DOUBLE: - return toDouble(value); + return toDouble(value, fieldName); case FLOAT: - return toFloat(value); + return toFloat(value, fieldName); case INT: - return toInteger(value); + return toInteger(value, fieldName); case LONG: - return toLong(value); + return toLong(value, fieldName); case SHORT: - return toShort(value); + return toShort(value, fieldName); case STRING: return toString(value, dateFormat, timeFormat, timestampFormat); case TIME: - return toTime(value, timeFormat); + return toTime(value, timeFormat, fieldName); case TIMESTAMP: - return toTimestamp(value, timestampFormat); + return toTimestamp(value, timestampFormat, fieldName); case ARRAY: - return toArray(value); + return toArray(value, fieldName); + case MAP: + return toMap(value, fieldName); case RECORD: final RecordDataType recordType = (RecordDataType) dataType; final RecordSchema childSchema = recordType.getChildSchema(); - return toRecord(value, childSchema); + return toRecord(value, childSchema, fieldName); case CHOICE: { if (value == null) { return null; @@ -89,10 +91,10 @@ public static Object convertType(final Object value, final DataType dataType, fi final DataType chosenDataType = chooseDataType(value, choiceDataType); if (chosenDataType == null) { throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() - + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes()); + + " for field " + fieldName + " to any of the following available Sub-Types for a Choice: " + choiceDataType.getPossibleSubTypes()); } - return convertType(value, chosenDataType); + return convertType(value, chosenDataType, fieldName); } } @@ -132,6 +134,8 @@ public static boolean isCompatibleDataType(final Object value, final DataType da return isTimestampTypeCompatible(value, dataType.getFormat()); case STRING: return isStringTypeCompatible(value); + case MAP: + return isMapTypeCompatible(value); case CHOICE: { final DataType chosenDataType = chooseDataType(value, (ChoiceDataType) dataType); return chosenDataType != null; @@ -151,7 +155,7 @@ public static DataType chooseDataType(final Object value, final ChoiceDataType c return null; } - public static Record toRecord(final Object value, final RecordSchema recordSchema) { + public static Record toRecord(final Object value, final RecordSchema recordSchema, final String fieldName) { if (value == null) { return null; } @@ -163,7 +167,7 @@ public static Record toRecord(final Object value, final RecordSchema recordSchem if (value instanceof Map) { if (recordSchema == null) { throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() - + " to Record because the value is a Map but no Record Schema was provided"); + + " to Record for field " + fieldName + " because the value is a Map but no Record Schema was provided"); } final Map map = (Map) value; @@ -182,21 +186,21 @@ public static Record toRecord(final Object value, final RecordSchema recordSchem } final Object rawValue = entry.getValue(); - final Object coercedValue = convertType(rawValue, desiredTypeOption.get()); + final Object coercedValue = convertType(rawValue, desiredTypeOption.get(), fieldName); coercedValues.put(key, coercedValue); } return new MapRecord(recordSchema, coercedValues); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Record for field " + fieldName); } public static boolean isRecordTypeCompatible(final Object value) { return value != null && value instanceof Record; } - public static Object[] toArray(final Object value) { + public static Object[] toArray(final Object value, final String fieldName) { if (value == null) { return null; } @@ -205,13 +209,70 @@ public static Object[] toArray(final Object value) { return (Object[]) value; } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Object Array"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Object Array for field " + fieldName); } public static boolean isArrayTypeCompatible(final Object value) { return value != null && value instanceof Object[]; } + @SuppressWarnings("unchecked") + public static Map toMap(final Object value, final String fieldName) { + if (value == null) { + return null; + } + + if (value instanceof Map) { + final Map original = (Map) value; + + boolean keysAreStrings = true; + for (final Object key : original.keySet()) { + if (!(key instanceof String)) { + keysAreStrings = false; + } + } + + if (keysAreStrings) { + return (Map) value; + } + + final Map transformed = new HashMap<>(); + for (final Map.Entry entry : original.entrySet()) { + final Object key = entry.getKey(); + if (key == null) { + transformed.put(null, entry.getValue()); + } else { + transformed.put(key.toString(), entry.getValue()); + } + } + + return transformed; + } + + if (value instanceof Record) { + final Record record = (Record) value; + final RecordSchema recordSchema = record.getSchema(); + if (recordSchema == null) { + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type Record to Map for field " + fieldName + + " because Record does not have an associated Schema"); + } + + final Map map = new HashMap<>(); + for (final String recordFieldName : recordSchema.getFieldNames()) { + map.put(recordFieldName, record.getValue(recordFieldName)); + } + + return map; + } + + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Map for field " + fieldName); + } + + public static boolean isMapTypeCompatible(final Object value) { + return value != null && value instanceof Map; + } + + public static String toString(final Object value, final String dateFormat, final String timeFormat, final String timestampFormat) { if (value == null) { return null; @@ -241,7 +302,7 @@ public static boolean isStringTypeCompatible(final Object value) { return value != null && (value instanceof String || value instanceof java.util.Date); } - public static java.sql.Date toDate(final Object value, final String format) { + public static java.sql.Date toDate(final Object value, final String format, final String fieldName) { if (value == null) { return null; } @@ -261,11 +322,11 @@ public static java.sql.Date toDate(final Object value, final String format) { return new Date(utilDate.getTime()); } catch (final ParseException e) { throw new IllegalTypeConversionException("Could not convert value [" + value - + "] of type java.lang.String to Date because the value is not in the expected date format: " + format); + + "] of type java.lang.String to Date because the value is not in the expected date format: " + format + " for field " + fieldName); } } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Date for field " + fieldName); } public static boolean isDateTypeCompatible(final Object value, final String format) { @@ -289,7 +350,7 @@ public static boolean isDateTypeCompatible(final Object value, final String form return false; } - public static Time toTime(final Object value, final String format) { + public static Time toTime(final Object value, final String format, final String fieldName) { if (value == null) { return null; } @@ -309,11 +370,11 @@ public static Time toTime(final Object value, final String format) { return new Time(utilDate.getTime()); } catch (final ParseException e) { throw new IllegalTypeConversionException("Could not convert value [" + value - + "] of type java.lang.String to Time because the value is not in the expected date format: " + format); + + "] of type java.lang.String to Time for field " + fieldName + " because the value is not in the expected date format: " + format); } } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Time for field " + fieldName); } private static DateFormat getDateFormat(final String format) { @@ -326,7 +387,7 @@ public static boolean isTimeTypeCompatible(final Object value, final String form return isDateTypeCompatible(value, format); } - public static Timestamp toTimestamp(final Object value, final String format) { + public static Timestamp toTimestamp(final Object value, final String format, final String fieldName) { if (value == null) { return null; } @@ -346,11 +407,11 @@ public static Timestamp toTimestamp(final Object value, final String format) { return new Timestamp(utilDate.getTime()); } catch (final ParseException e) { throw new IllegalTypeConversionException("Could not convert value [" + value - + "] of type java.lang.String to Timestamp because the value is not in the expected date format: " + format); + + "] of type java.lang.String to Timestamp for field " + fieldName + " because the value is not in the expected date format: " + format); } } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Timestamp for field " + fieldName); } public static boolean isTimestampTypeCompatible(final Object value, final String format) { @@ -358,7 +419,7 @@ public static boolean isTimestampTypeCompatible(final Object value, final String } - public static BigInteger toBigInt(final Object value) { + public static BigInteger toBigInt(final Object value, final String fieldName) { if (value == null) { return null; } @@ -370,14 +431,14 @@ public static BigInteger toBigInt(final Object value) { return BigInteger.valueOf((Long) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger for field " + fieldName); } public static boolean isBigIntTypeCompatible(final Object value) { return value == null && (value instanceof BigInteger || value instanceof Long); } - public static Boolean toBoolean(final Object value) { + public static Boolean toBoolean(final Object value, final String fieldName) { if (value == null) { return null; } @@ -394,7 +455,7 @@ public static Boolean toBoolean(final Object value) { } } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Boolean"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Boolean for field " + fieldName); } public static boolean isBooleanTypeCompatible(final Object value) { @@ -411,7 +472,7 @@ public static boolean isBooleanTypeCompatible(final Object value) { return false; } - public static Double toDouble(final Object value) { + public static Double toDouble(final Object value, final String fieldName) { if (value == null) { return null; } @@ -424,7 +485,7 @@ public static Double toDouble(final Object value) { return Double.parseDouble((String) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Double"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Double for field " + fieldName); } public static boolean isDoubleTypeCompatible(final Object value) { @@ -452,7 +513,7 @@ private static boolean isNumberTypeCompatible(final Object value, final Consumer return false; } - public static Float toFloat(final Object value) { + public static Float toFloat(final Object value, final String fieldName) { if (value == null) { return null; } @@ -465,14 +526,14 @@ public static Float toFloat(final Object value) { return Float.parseFloat((String) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Float"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Float for field " + fieldName); } public static boolean isFloatTypeCompatible(final Object value) { return isNumberTypeCompatible(value, s -> Float.parseFloat(s)); } - public static Long toLong(final Object value) { + public static Long toLong(final Object value, final String fieldName) { if (value == null) { return null; } @@ -489,7 +550,7 @@ public static Long toLong(final Object value) { return ((java.util.Date) value).getTime(); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Long"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Long for field " + fieldName); } public static boolean isLongTypeCompatible(final Object value) { @@ -518,7 +579,7 @@ public static boolean isLongTypeCompatible(final Object value) { } - public static Integer toInteger(final Object value) { + public static Integer toInteger(final Object value, final String fieldName) { if (value == null) { return null; } @@ -531,7 +592,7 @@ public static Integer toInteger(final Object value) { return Integer.parseInt((String) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Integer"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Integer for field " + fieldName); } public static boolean isIntegerTypeCompatible(final Object value) { @@ -539,7 +600,7 @@ public static boolean isIntegerTypeCompatible(final Object value) { } - public static Short toShort(final Object value) { + public static Short toShort(final Object value, final String fieldName) { if (value == null) { return null; } @@ -552,14 +613,14 @@ public static Short toShort(final Object value) { return Short.parseShort((String) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Short"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Short for field " + fieldName); } public static boolean isShortTypeCompatible(final Object value) { return isNumberTypeCompatible(value, s -> Short.parseShort(s)); } - public static Byte toByte(final Object value) { + public static Byte toByte(final Object value, final String fieldName) { if (value == null) { return null; } @@ -572,7 +633,7 @@ public static Byte toByte(final Object value) { return Byte.parseByte((String) value); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Byte"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Byte for field " + fieldName); } public static boolean isByteTypeCompatible(final Object value) { @@ -580,7 +641,7 @@ public static boolean isByteTypeCompatible(final Object value) { } - public static Character toCharacter(final Object value) { + public static Character toCharacter(final Object value, final String fieldName) { if (value == null) { return null; } @@ -592,13 +653,14 @@ public static Character toCharacter(final Object value) { if (value instanceof CharSequence) { final CharSequence charSeq = (CharSequence) value; if (charSeq.length() == 0) { - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character because it has a length of 0"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + + " to Character because it has a length of 0 for field " + fieldName); } return charSeq.charAt(0); } - throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character"); + throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to Character for field " + fieldName); } public static boolean isCharacterTypeCompatible(final Object value) { diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml index d86a8c5a4f75..16479f185afe 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/pom.xml @@ -70,7 +70,11 @@ org.apache.avro avro - 1.8.1 + + + org.apache.nifi + nifi-mock + test @@ -96,6 +100,7 @@ src/test/resources/json/json-with-unicode.json src/test/resources/json/primitive-type-array.json src/test/resources/json/single-bank-account.json + src/test/resources/json/single-bank-account-wrong-field-type.json src/test/resources/json/single-element-nested-array.json src/test/resources/json/single-element-nested.json src/test/resources/json/output/dataTypes.json diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java index f92816f742b3..8b5944b3a7ce 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java @@ -19,31 +19,47 @@ import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; -import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaRegistryService; @Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"}) @CapabilityDescription("Parses Avro data and returns each Avro record as an separate Record object. The Avro data must contain " + "the schema itself.") -public class AvroReader extends AbstractControllerService implements RowRecordReaderFactory { +public class AvroReader extends SchemaRegistryService implements RecordReaderFactory { + private final AllowableValue EMBEDDED_AVRO_SCHEMA = new AllowableValue("embedded-avro-schema", + "Use Embedded Avro Schema", "The FlowFile has the Avro Schema embedded within the content, and this schema will be used."); + @Override - public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException { - return new AvroRecordReader(in); + protected List getSchemaAccessStrategyValues() { + final List allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues()); + allowableValues.add(EMBEDDED_AVRO_SCHEMA); + return allowableValues; } @Override - public RecordSchema getSchema(final FlowFile flowFile) throws MalformedRecordException, IOException { - // TODO: Need to support retrieving schema from registry instead of requiring that it be in the Avro file. - return null; + public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws MalformedRecordException, IOException, SchemaNotFoundException { + final String schemaAccessStrategy = getConfigurationContext().getProperty(SCHEMA_ACCESS_STRATEGY).getValue(); + if (EMBEDDED_AVRO_SCHEMA.getValue().equals(schemaAccessStrategy)) { + return new AvroReaderWithEmbeddedSchema(in); + } else { + return new AvroReaderWithExplicitSchema(in, getSchema(flowFile, in)); + } } + @Override + protected AllowableValue getDefaultSchemaAccessStrategy() { + return EMBEDDED_AVRO_SCHEMA; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java new file mode 100644 index 000000000000..aa61e4cf2722 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithEmbeddedSchema.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.avro; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.record.RecordSchema; + +public class AvroReaderWithEmbeddedSchema extends AvroRecordReader { + private final DataFileStream dataFileStream; + private final InputStream in; + private final Schema avroSchema; + private final RecordSchema recordSchema; + + public AvroReaderWithEmbeddedSchema(final InputStream in) throws IOException { + this.in = in; + dataFileStream = new DataFileStream<>(in, new GenericDatumReader()); + this.avroSchema = dataFileStream.getSchema(); + recordSchema = AvroTypeUtil.createSchema(avroSchema); + } + + @Override + public void close() throws IOException { + dataFileStream.close(); + in.close(); + } + + @Override + protected GenericRecord nextAvroRecord() { + if (!dataFileStream.hasNext()) { + return null; + } + + return dataFileStream.next(); + } + + @Override + public RecordSchema getSchema() throws MalformedRecordException { + return recordSchema; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java new file mode 100644 index 000000000000..104214cf8753 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReaderWithExplicitSchema.java @@ -0,0 +1,75 @@ +/* + * 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.avro; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.record.RecordSchema; + +public class AvroReaderWithExplicitSchema extends AvroRecordReader { + private final InputStream in; + private final Schema avroSchema; + private final RecordSchema recordSchema; + private final DatumReader datumReader; + private final BinaryDecoder decoder; + private GenericRecord genericRecord; + + public AvroReaderWithExplicitSchema(final InputStream in, final RecordSchema recordSchema) throws IOException, SchemaNotFoundException { + this.in = in; + this.recordSchema = recordSchema; + + this.avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema); + datumReader = new GenericDatumReader(avroSchema); + decoder = DecoderFactory.get().binaryDecoder(in, null); + } + + @Override + public void close() throws IOException { + in.close(); + } + + @Override + protected GenericRecord nextAvroRecord() throws IOException { + if (decoder.isEnd()) { + return null; + } + + try { + genericRecord = datumReader.read(genericRecord, decoder); + } catch (final EOFException eof) { + return null; + } + + return genericRecord; + } + + @Override + public RecordSchema getSchema() throws MalformedRecordException { + return recordSchema; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java index d725cbfdc6f8..50fcaf7dfd30 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java @@ -18,7 +18,6 @@ package org.apache.nifi.avro; import java.io.IOException; -import java.io.InputStream; import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.HashMap; @@ -30,11 +29,8 @@ import org.apache.avro.LogicalTypes; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; -import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericData.Array; -import org.apache.avro.generic.GenericData.StringType; -import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericFixed; import org.apache.avro.generic.GenericRecord; import org.apache.avro.util.Utf8; @@ -48,38 +44,19 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.util.DataTypeUtils; -public class AvroRecordReader implements RecordReader { - private final InputStream in; - private final Schema avroSchema; - private final DataFileStream dataFileStream; - private RecordSchema recordSchema; +public abstract class AvroRecordReader implements RecordReader { - public AvroRecordReader(final InputStream in) throws IOException, MalformedRecordException { - this.in = in; + protected abstract GenericRecord nextAvroRecord() throws IOException; - dataFileStream = new DataFileStream<>(in, new GenericDatumReader()); - this.avroSchema = dataFileStream.getSchema(); - GenericData.setStringType(this.avroSchema, StringType.String); - } - - @Override - public void close() throws IOException { - dataFileStream.close(); - in.close(); - } @Override public Record nextRecord() throws IOException, MalformedRecordException { - if (!dataFileStream.hasNext()) { + GenericRecord record = nextAvroRecord(); + if (record == null) { return null; } - GenericRecord record = null; - while (record == null && dataFileStream.hasNext()) { - record = dataFileStream.next(); - } - final RecordSchema schema = getSchema(); final Map values = convertAvroRecordToMap(record, schema); return new MapRecord(schema, values); @@ -102,7 +79,7 @@ private Map convertAvroRecordToMap(final GenericRecord avroRecor final Object rawValue = normalizeValue(value, fieldSchema); final DataType desiredType = recordSchema.getDataType(fieldName).get(); - final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType); + final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType, fieldName); values.put(fieldName, coercedValue); } @@ -215,13 +192,5 @@ private Object normalizeValue(final Object value, final Schema avroSchema) { } - @Override - public RecordSchema getSchema() throws MalformedRecordException { - if (recordSchema != null) { - return recordSchema; - } - recordSchema = AvroTypeUtil.createSchema(avroSchema); - return recordSchema; - } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java index 03d766cbf95e..fc1482b9bec2 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java @@ -17,48 +17,78 @@ package org.apache.nifi.avro; +import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.List; import org.apache.avro.Schema; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.SchemaRegistryRecordSetWriter; +import org.apache.nifi.serialization.record.RecordSchema; @Tags({"avro", "result", "set", "writer", "serializer", "record", "recordset", "row"}) @CapabilityDescription("Writes the contents of a RecordSet in Binary Avro format.") -public class AvroRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory { - static final PropertyDescriptor SCHEMA = new PropertyDescriptor.Builder() - .name("Avro Schema") - .description("The Avro Schema to use when writing out the Result Set") - .addValidator(new AvroSchemaValidator()) - .expressionLanguageSupported(false) - .required(true) +public class AvroRecordSetWriter extends SchemaRegistryRecordSetWriter implements RecordSetWriterFactory { + + static final AllowableValue AVRO_EMBEDDED = new AllowableValue("avro-embedded", "Embed Avro Schema", + "The FlowFile will have the Avro schema embedded into the content, as is typical with Avro"); + + protected static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder() + .name("Schema Registry") + .description("Specifies the Controller Service to use for the Schema Registry") + .identifiesControllerService(SchemaRegistry.class) + .required(false) .build(); - private volatile Schema schema; @Override protected List getSupportedPropertyDescriptors() { - final List properties = new ArrayList<>(); - properties.add(SCHEMA); + final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + properties.add(SCHEMA_ACCESS_STRATEGY); + properties.add(SCHEMA_REGISTRY); return properties; } - @OnEnabled - public void storePropertyValues(final ConfigurationContext context) { - schema = new Schema.Parser().parse(context.getProperty(SCHEMA).getValue()); + + @Override + public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws IOException { + final String strategyValue = getConfigurationContext().getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + + try { + final RecordSchema recordSchema = getSchema(flowFile, in); + final Schema avroSchema = AvroTypeUtil.extractAvroSchema(recordSchema); + + if (AVRO_EMBEDDED.getValue().equals(strategyValue)) { + return new WriteAvroResultWithSchema(avroSchema); + } else { + return new WriteAvroResultWithExternalSchema(avroSchema, recordSchema, getSchemaAccessWriter()); + } + } catch (final SchemaNotFoundException e) { + throw new ProcessException("Could not determine the Avro Schema to use for writing the content", e); + } } @Override - public RecordSetWriter createWriter(final ComponentLog logger) { - return new WriteAvroResult(schema); + protected List getSchemaWriteStrategyValues() { + final List allowableValues = new ArrayList<>(); + allowableValues.add(AVRO_EMBEDDED); + allowableValues.addAll(super.getSchemaWriteStrategyValues()); + return allowableValues; } + @Override + protected AllowableValue getDefaultSchemaWriteStrategy() { + return AVRO_EMBEDDED; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java index 7151348f7e4e..4449afc033a4 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroSchemaValidator.java @@ -26,6 +26,15 @@ public class AvroSchemaValidator implements Validator { @Override public ValidationResult validate(final String subject, final String input, final ValidationContext context) { + if (context.isExpressionLanguageSupported(subject) && context.isExpressionLanguagePresent(input)) { + return new ValidationResult.Builder() + .input(input) + .subject(subject) + .valid(true) + .explanation("Expression Language is present") + .build(); + } + try { new Schema.Parser().parse(input); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java index 1810c837d30a..db5f29dfb4d4 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java @@ -20,6 +20,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.stream.Collectors; import org.apache.avro.LogicalType; @@ -27,14 +28,37 @@ import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.Schema.Type; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.SimpleRecordSchema; 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.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; import org.apache.nifi.serialization.record.util.IllegalTypeConversionException; public class AvroTypeUtil { + public static final String AVRO_SCHEMA_FORMAT = "avro"; + + public static Schema extractAvroSchema(final RecordSchema recordSchema) throws SchemaNotFoundException { + final Optional schemaFormatOption = recordSchema.getSchemaFormat(); + if (!schemaFormatOption.isPresent()) { + throw new SchemaNotFoundException("No Schema Format was present in the RecordSchema"); + } + + final String schemaFormat = schemaFormatOption.get(); + if (!schemaFormat.equals(AVRO_SCHEMA_FORMAT)) { + throw new SchemaNotFoundException("Schema provided is not in Avro format"); + } + + final Optional textOption = recordSchema.getSchemaText(); + if (!textOption.isPresent()) { + throw new SchemaNotFoundException("No Schema text was present in the RecordSchema"); + } + + final String text = textOption.get(); + return new Schema.Parser().parse(text); + } public static DataType determineDataType(final Schema avroSchema) { final Type avroType = avroSchema.getType(); @@ -96,12 +120,15 @@ public static DataType determineDataType(final Schema avroSchema) { recordFields.add(new RecordField(fieldName, fieldType)); } - final RecordSchema recordSchema = new SimpleRecordSchema(recordFields); + final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY); return RecordFieldType.RECORD.getRecordDataType(recordSchema); } case NULL: + return RecordFieldType.STRING.getDataType(); case MAP: - return RecordFieldType.RECORD.getDataType(); + final Schema valueSchema = avroSchema.getValueType(); + final DataType valueType = determineDataType(valueSchema); + return RecordFieldType.MAP.getMapDataType(valueType); case UNION: { final List nonNullSubSchemas = avroSchema.getTypes().stream() .filter(s -> s.getType() != Type.NULL) @@ -132,7 +159,7 @@ public static RecordSchema createSchema(final Schema avroSchema) { recordFields.add(new RecordField(fieldName, dataType)); } - final RecordSchema recordSchema = new SimpleRecordSchema(recordFields); + final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY); return recordSchema; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java index b512b8200d91..1c0b287685d6 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java @@ -44,40 +44,21 @@ import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; -import org.apache.nifi.serialization.record.RecordSet; import org.apache.nifi.serialization.record.util.DataTypeUtils; import org.apache.nifi.serialization.record.util.IllegalTypeConversionException; -public class WriteAvroResult implements RecordSetWriter { +public abstract class WriteAvroResult implements RecordSetWriter { private final Schema schema; public WriteAvroResult(final Schema schema) { this.schema = schema; } - @Override - public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException { - Record record = rs.next(); - if (record == null) { - return WriteResult.of(0, Collections.emptyMap()); - } - - int nrOfRows = 0; - final DatumWriter datumWriter = new GenericDatumWriter<>(schema); - try (final DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter)) { - dataFileWriter.create(schema, outStream); - - do { - final GenericRecord rec = createAvroRecord(record, schema); - dataFileWriter.append(rec); - nrOfRows++; - } while ((record = rs.next()) != null); - } - - return WriteResult.of(nrOfRows, Collections.emptyMap()); + protected Schema getSchema() { + return schema; } - private GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException { + protected GenericRecord createAvroRecord(final Record record, final Schema avroSchema) throws IOException { final GenericRecord rec = new GenericData.Record(avroSchema); final RecordSchema recordSchema = record.getSchema(); @@ -89,14 +70,14 @@ private GenericRecord createAvroRecord(final Record record, final Schema avroSch continue; } - final Object converted = convertToAvroObject(rawValue, field.schema()); + final Object converted = convertToAvroObject(rawValue, field.schema(), fieldName); rec.put(fieldName, converted); } return rec; } - private Object convertToAvroObject(final Object rawValue, final Schema fieldSchema) throws IOException { + protected Object convertToAvroObject(final Object rawValue, final Schema fieldSchema, final String fieldName) throws IOException { if (rawValue == null) { return null; } @@ -105,43 +86,43 @@ private Object convertToAvroObject(final Object rawValue, final Schema fieldSche case INT: { final LogicalType logicalType = fieldSchema.getLogicalType(); if (logicalType == null) { - return DataTypeUtils.toInteger(rawValue); + return DataTypeUtils.toInteger(rawValue, fieldName); } if (LogicalTypes.date().getName().equals(logicalType.getName())) { - final long longValue = DataTypeUtils.toLong(rawValue); + final long longValue = DataTypeUtils.toLong(rawValue, fieldName); final Date date = new Date(longValue); final Duration duration = Duration.between(new Date(0L).toInstant(), date.toInstant()); final long days = duration.toDays(); return (int) days; } else if (LogicalTypes.timeMillis().getName().equals(logicalType.getName())) { - final long longValue = DataTypeUtils.toLong(rawValue); + final long longValue = DataTypeUtils.toLong(rawValue, fieldName); final Date date = new Date(longValue); final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant()); final long millisSinceMidnight = duration.toMillis(); return (int) millisSinceMidnight; } - return DataTypeUtils.toInteger(rawValue); + return DataTypeUtils.toInteger(rawValue, fieldName); } case LONG: { final LogicalType logicalType = fieldSchema.getLogicalType(); if (logicalType == null) { - return DataTypeUtils.toLong(rawValue); + return DataTypeUtils.toLong(rawValue, fieldName); } if (LogicalTypes.timeMicros().getName().equals(logicalType.getName())) { - final long longValue = DataTypeUtils.toLong(rawValue); + final long longValue = DataTypeUtils.toLong(rawValue, fieldName); final Date date = new Date(longValue); final Duration duration = Duration.between(date.toInstant().truncatedTo(ChronoUnit.DAYS), date.toInstant()); return duration.toMillis() * 1000L; } else if (LogicalTypes.timestampMillis().getName().equals(logicalType.getName())) { - return DataTypeUtils.toLong(rawValue); + return DataTypeUtils.toLong(rawValue, fieldName); } else if (LogicalTypes.timestampMicros().getName().equals(logicalType.getName())) { - return DataTypeUtils.toLong(rawValue) * 1000L; + return DataTypeUtils.toLong(rawValue, fieldName) * 1000L; } - return DataTypeUtils.toLong(rawValue); + return DataTypeUtils.toLong(rawValue, fieldName); } case BYTES: case FIXED: @@ -180,7 +161,7 @@ private Object convertToAvroObject(final Object rawValue, final Schema fieldSche continue; } - final Object converted = convertToAvroObject(recordFieldValue, field.schema()); + final Object converted = convertToAvroObject(recordFieldValue, field.schema(), fieldName); avroRecord.put(recordFieldName, converted); } return avroRecord; @@ -188,16 +169,16 @@ private Object convertToAvroObject(final Object rawValue, final Schema fieldSche final Object[] objectArray = (Object[]) rawValue; final List list = new ArrayList<>(objectArray.length); for (final Object o : objectArray) { - final Object converted = convertToAvroObject(o, fieldSchema.getElementType()); + final Object converted = convertToAvroObject(o, fieldSchema.getElementType(), fieldName); list.add(converted); } return list; case BOOLEAN: - return DataTypeUtils.toBoolean(rawValue); + return DataTypeUtils.toBoolean(rawValue, fieldName); case DOUBLE: - return DataTypeUtils.toDouble(rawValue); + return DataTypeUtils.toDouble(rawValue, fieldName); case FLOAT: - return DataTypeUtils.toFloat(rawValue); + return DataTypeUtils.toFloat(rawValue, fieldName); case NULL: return null; case ENUM: diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java new file mode 100644 index 000000000000..74306e4e84fd --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithExternalSchema.java @@ -0,0 +1,75 @@ +/* + * 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.avro; + +import java.io.BufferedOutputStream; +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryEncoder; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.io.EncoderFactory; +import org.apache.nifi.schema.access.SchemaAccessWriter; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.RecordSet; + +public class WriteAvroResultWithExternalSchema extends WriteAvroResult { + private final SchemaAccessWriter schemaAccessWriter; + private final RecordSchema recordSchema; + + public WriteAvroResultWithExternalSchema(final Schema avroSchema, final RecordSchema recordSchema, final SchemaAccessWriter schemaAccessWriter) { + super(avroSchema); + this.recordSchema = recordSchema; + this.schemaAccessWriter = schemaAccessWriter; + } + + @Override + public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException { + Record record = rs.next(); + if (record == null) { + return WriteResult.of(0, Collections.emptyMap()); + } + + int nrOfRows = 0; + final Schema schema = getSchema(); + final DatumWriter datumWriter = new GenericDatumWriter<>(schema); + + final BufferedOutputStream bufferedOut = new BufferedOutputStream(outStream); + schemaAccessWriter.writeHeader(recordSchema, bufferedOut); + + final BinaryEncoder encoder = EncoderFactory.get().blockingBinaryEncoder(bufferedOut, null); + + do { + final GenericRecord rec = createAvroRecord(record, schema); + + datumWriter.write(rec, encoder); + encoder.flush(); + nrOfRows++; + } while ((record = rs.next()) != null); + + bufferedOut.flush(); + + return WriteResult.of(nrOfRows, schemaAccessWriter.getAttributes(recordSchema)); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java new file mode 100644 index 000000000000..dca8aac86ffa --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResultWithSchema.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.avro; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericDatumWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.DatumWriter; +import org.apache.nifi.serialization.WriteResult; +import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordSet; + +public class WriteAvroResultWithSchema extends WriteAvroResult { + + public WriteAvroResultWithSchema(final Schema schema) { + super(schema); + } + + @Override + public WriteResult write(final RecordSet rs, final OutputStream outStream) throws IOException { + Record record = rs.next(); + if (record == null) { + return WriteResult.of(0, Collections.emptyMap()); + } + + int nrOfRows = 0; + final Schema schema = getSchema(); + final DatumWriter datumWriter = new GenericDatumWriter<>(schema); + + try (final DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter)) { + dataFileWriter.create(schema, outStream); + + do { + final GenericRecord rec = createAvroRecord(record, schema); + dataFileWriter.append(rec); + nrOfRows++; + } while ((record = rs.next()) != null); + } + + return WriteResult.of(nrOfRows, Collections.emptyMap()); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java new file mode 100644 index 000000000000..9ad3b8cdbbf0 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java @@ -0,0 +1,59 @@ +/* + * 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.csv; + +import java.io.InputStream; +import java.io.InputStreamReader; +import java.io.Reader; +import java.util.ArrayList; +import java.util.List; + +import org.apache.commons.csv.CSVFormat; +import org.apache.commons.csv.CSVParser; +import org.apache.commons.io.input.BOMInputStream; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; + +public class CSVHeaderSchemaStrategy implements SchemaAccessStrategy { + + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { + try { + final CSVFormat csvFormat = CSVUtils.createCSVFormat(context).withFirstRecordAsHeader(); + try (final Reader reader = new InputStreamReader(new BOMInputStream(contentStream)); + final CSVParser csvParser = new CSVParser(reader, csvFormat)) { + + final List fields = new ArrayList<>(); + for (final String columnName : csvParser.getHeaderMap().keySet()) { + fields.add(new RecordField(columnName, RecordFieldType.STRING.getDataType())); + } + + return new SimpleRecordSchema(fields); + } + } catch (final Exception e) { + throw new SchemaNotFoundException("Failed to read Header line from CSV", e); + } + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java index 6b06ebf65707..789b1eb1b62b 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java @@ -17,6 +17,7 @@ package org.apache.nifi.csv; +import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -26,21 +27,31 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; import org.apache.nifi.serialization.DateTimeUtils; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; -import org.apache.nifi.serialization.SchemaRegistryRecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaRegistryService; import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.stream.io.NonCloseableInputStream; @Tags({"csv", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"}) @CapabilityDescription("Parses CSV-formatted data, returning each row in the CSV file as a separate record. " + "This reader assumes that the first line in the content is the column names and all subsequent lines are " + "the values. See Controller Service's Usage for further documentation.") -public class CSVReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory { +public class CSVReader extends SchemaRegistryService implements RecordReaderFactory { + + private final AllowableValue headerDerivedAllowableValue = new AllowableValue("csv-header-derived", "Use String Fields From Header", + "The first non-comment line of the CSV file is a header line that contains the names of the columns. The schema will be derived by using the " + + "column names in the header and assuming that all columns are of type String."); + private final SchemaAccessStrategy headerDerivedSchemaStrategy = new CSVHeaderSchemaStrategy(); private volatile CSVFormat csvFormat; private volatile String dateFormat; @@ -73,9 +84,29 @@ public void storeCsvFormat(final ConfigurationContext context) { } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException { - final RecordSchema schema = getSchema(flowFile); - return new CSVRecordReader(in, logger, schema, csvFormat, dateFormat, timeFormat, timestampFormat); + public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + // Use Mark/Reset of a BufferedInputStream in case we read from the Input Stream for the header. + final BufferedInputStream bufferedIn = new BufferedInputStream(in); + bufferedIn.mark(1024 * 1024); + final RecordSchema schema = getSchema(flowFile, new NonCloseableInputStream(bufferedIn)); + bufferedIn.reset(); + + return new CSVRecordReader(bufferedIn, logger, schema, csvFormat, dateFormat, timeFormat, timestampFormat); + } + + @Override + protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) { + if (allowableValue.equalsIgnoreCase(headerDerivedAllowableValue.getValue())) { + return headerDerivedSchemaStrategy; + } + + return super.getSchemaAccessStrategy(allowableValue, schemaRegistry); } + @Override + protected List getSchemaAccessStrategyValues() { + final List allowableValues = new ArrayList<>(super.getSchemaAccessStrategyValues()); + allowableValues.add(headerDerivedAllowableValue); + return allowableValues; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java index d02768c84974..161624551fd7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java @@ -48,6 +48,7 @@ public class CSVRecordReader implements RecordReader { public CSVRecordReader(final InputStream in, final ComponentLog logger, final RecordSchema schema, final CSVFormat csvFormat, final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException { + // TODO: Need to make sure that we use correct logic for skipping header line. final Reader reader = new InputStreamReader(new BOMInputStream(in)); csvParser = new CSVParser(reader, csvFormat); @@ -71,7 +72,7 @@ public Record nextRecord() throws IOException, MalformedRecordException { continue; } - final Object converted = convert(rawValue, schema.getDataType(fieldName).orElse(null)); + final Object converted = convert(rawValue, schema.getDataType(fieldName).orElse(null), fieldName); rowValues.put(fieldName, converted); } @@ -86,7 +87,7 @@ public RecordSchema getSchema() { return schema; } - protected Object convert(final String value, final DataType dataType) { + protected Object convert(final String value, final DataType dataType, final String fieldName) { if (dataType == null || value == null) { return value; } @@ -97,7 +98,7 @@ protected Object convert(final String value, final DataType dataType) { return null; } - return DataTypeUtils.convertType(trimmed, dataType, dateFormat, timeFormat, timestampFormat); + return DataTypeUtils.convertType(trimmed, dataType, dateFormat, timeFormat, timestampFormat, fieldName); } @Override diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java index 6a7b758fb8e6..5d22afb77a94 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java @@ -17,6 +17,8 @@ package org.apache.nifi.csv; +import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.List; @@ -26,10 +28,13 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.DateTimeTextRecordSetWriter; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.RecordSchema; @Tags({"csv", "result", "set", "recordset", "record", "writer", "serializer", "row", "tsv", "tab", "separated", "delimited"}) @CapabilityDescription("Writes the contents of a RecordSet as CSV data. The first line written " @@ -60,8 +65,8 @@ public void storeCsvFormat(final ConfigurationContext context) { } @Override - public RecordSetWriter createWriter(final ComponentLog logger) { - return new WriteCSVResult(csvFormat, getDateFormat(), getTimeFormat(), getTimestampFormat()); + public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws SchemaNotFoundException, IOException { + final RecordSchema schema = getSchema(flowFile, in); + return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(), getDateFormat(), getTimeFormat(), getTimestampFormat()); } - } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java index e23b6e1c988a..c63a765d5266 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java @@ -157,6 +157,7 @@ private static CSVFormat buildCustomFormat(final ConfigurationContext context) { CSVFormat format = CSVFormat.newFormat(valueSeparator) .withAllowMissingColumnNames() .withIgnoreEmptyLines() + // TODO: Need property to specify whether or not to skip first line as header! .withFirstRecordAsHeader(); format = format.withQuote(getChar(context, QUOTE_CHAR)); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java index e0eb8134c211..a3f15bd8212e 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java @@ -25,6 +25,7 @@ import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVPrinter; +import org.apache.nifi.schema.access.SchemaAccessWriter; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.DataType; @@ -35,12 +36,17 @@ public class WriteCSVResult implements RecordSetWriter { private final CSVFormat csvFormat; + private final RecordSchema recordSchema; + private final SchemaAccessWriter schemaWriter; private final String dateFormat; private final String timeFormat; private final String timestampFormat; - public WriteCSVResult(final CSVFormat csvFormat, final String dateFormat, final String timeFormat, final String timestampFormat) { + public WriteCSVResult(final CSVFormat csvFormat, final RecordSchema recordSchema, final SchemaAccessWriter schemaWriter, + final String dateFormat, final String timeFormat, final String timestampFormat) { this.csvFormat = csvFormat; + this.recordSchema = recordSchema; + this.schemaWriter = schemaWriter; this.dateFormat = dateFormat; this.timeFormat = timeFormat; this.timestampFormat = timestampFormat; @@ -69,9 +75,10 @@ private String getFormat(final Record record, final String fieldName) { public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException { int count = 0; - final RecordSchema schema = rs.getSchema(); - final String[] columnNames = schema.getFieldNames().toArray(new String[0]); - final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames); + final String[] columnNames = recordSchema.getFieldNames().toArray(new String[0]); + final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames).withSkipHeaderRecord(false); + + schemaWriter.writeHeader(recordSchema, rawOut); try (final OutputStream nonCloseable = new NonCloseableOutputStream(rawOut); final OutputStreamWriter streamWriter = new OutputStreamWriter(nonCloseable); @@ -80,9 +87,9 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I try { Record record; while ((record = rs.next()) != null) { - final Object[] colVals = new Object[schema.getFieldCount()]; + final Object[] colVals = new Object[recordSchema.getFieldCount()]; int i = 0; - for (final String fieldName : schema.getFieldNames()) { + for (final String fieldName : recordSchema.getFieldNames()) { colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName)); } @@ -94,7 +101,7 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I } } - return WriteResult.of(count, Collections.emptyMap()); + return WriteResult.of(count, schemaWriter.getAttributes(recordSchema)); } @Override diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java index f444b8a90f44..6af92294369b 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java @@ -23,38 +23,58 @@ import java.io.Reader; import java.util.ArrayList; import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; -import org.apache.nifi.serialization.SchemaRegistryRecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaRegistryService; +import org.apache.nifi.serialization.SimpleRecordSchema; +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.RecordSchema; import io.thekraken.grok.api.Grok; +import io.thekraken.grok.api.GrokUtils; import io.thekraken.grok.api.exception.GrokException; @Tags({"grok", "logs", "logfiles", "parse", "unstructured", "text", "record", "reader", "regex", "pattern", "logstash"}) @CapabilityDescription("Provides a mechanism for reading unstructured text data, such as log files, and structuring the data " + "so that it can be processed. The service is configured using Grok patterns. " + "The service reads from a stream of data and splits each message that it finds into a separate Record, each containing the fields that are configured. " - + "If a line in the input does not match the expected message pattern, the line of text is considered to be part of the previous " - + "message, with the exception of stack traces. A stack trace that is found at the end of a log message is considered to be part " - + "of the previous message but is added to the 'STACK_TRACE' field of the Record. If a record has no stack trace, it will have a NULL value " - + "for the STACK_TRACE field. All fields that are parsed are considered to be of type String by default. If there is need to change the type of a field, " - + "this can be accomplished by configuring the Schema Registry to use and adding the appropriate schema.") -public class GrokReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory { + + "If a line in the input does not match the expected message pattern, the line of text is either considered to be part of the previous " + + "message or is skipped, depending on the configuration,, with the exception of stack traces. A stack trace that is found at the end of " + + "a log message is considered to be part of the previous message but is added to the 'stackTrace' field of the Record. If a record has " + + "no stack trace, it will have a NULL value for the stackTrace field. All fields that are parsed are considered to be of type String by default. " + + "If there is need to change the type of a field, this can be accomplished by configuring the Schema Registry to use and adding the appropriate schema.") +public class GrokReader extends SchemaRegistryService implements RecordReaderFactory { private volatile Grok grok; - private volatile boolean useSchemaRegistry; + private volatile boolean appendUnmatchedLine; + private volatile RecordSchema recordSchema; private static final String DEFAULT_PATTERN_NAME = "/default-grok-patterns.txt"; + static final AllowableValue APPEND_TO_PREVIOUS_MESSAGE = new AllowableValue("append-to-previous-message", "Append to Previous Message", + "The line of text that does not match the Grok Expression will be appended to the last field of the prior message."); + static final AllowableValue SKIP_LINE = new AllowableValue("skip-line", "Skip Line", + "The line of text that does not match the Grok Expression will be skipped."); + + static final AllowableValue STRING_FIELDS_FROM_GROK_EXPRESSION = new AllowableValue("string-fields-from-grok-expression", "Use String Fields From Grok Expression", + "The schema will be derived by using the field names present in the Grok Expression. All fields will be assumed to be of type String."); + static final PropertyDescriptor PATTERN_FILE = new PropertyDescriptor.Builder() .name("Grok Pattern File") .description("Path to a file that contains Grok Patterns to use for parsing logs. If not specified, a built-in default Pattern file " @@ -73,11 +93,22 @@ public class GrokReader extends SchemaRegistryRecordReader implements RowRecordR .required(true) .build(); + static final PropertyDescriptor NO_MATCH_BEHAVIOR = new PropertyDescriptor.Builder() + .name("no-match-behavior") + .displayName("No Match Behavior") + .description("If a line of text is encountered and it does not match the given Grok Expression, and it is not part of a stack trace, " + + "this property specifies how the text should be processed.") + .allowableValues(APPEND_TO_PREVIOUS_MESSAGE, SKIP_LINE) + .defaultValue(APPEND_TO_PREVIOUS_MESSAGE.getValue()) + .required(true) + .build(); + @Override protected List getSupportedPropertyDescriptors() { final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); properties.add(PATTERN_FILE); properties.add(GROK_EXPRESSION); + properties.add(NO_MATCH_BEHAVIOR); return properties; } @@ -95,17 +126,71 @@ public void preCompile(final ConfigurationContext context) throws GrokException, } grok.compile(context.getProperty(GROK_EXPRESSION).getValue()); - useSchemaRegistry = context.getProperty(OPTIONAL_SCHEMA_NAME).isSet() && context.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet(); + + appendUnmatchedLine = context.getProperty(NO_MATCH_BEHAVIOR).getValue().equalsIgnoreCase(APPEND_TO_PREVIOUS_MESSAGE.getValue()); + + this.recordSchema = createRecordSchema(grok); } + static RecordSchema createRecordSchema(final Grok grok) { + final List fields = new ArrayList<>(); + + String grokExpression = grok.getOriginalGrokPattern(); + while (grokExpression.length() > 0) { + final Matcher matcher = GrokUtils.GROK_PATTERN.matcher(grokExpression); + if (matcher.find()) { + final Map namedGroups = GrokUtils.namedGroups(matcher, grokExpression); + final String fieldName = namedGroups.get("subname"); + + DataType dataType = RecordFieldType.STRING.getDataType(); + final RecordField recordField = new RecordField(fieldName, dataType); + fields.add(recordField); + + if (grokExpression.length() > matcher.end() + 1) { + grokExpression = grokExpression.substring(matcher.end() + 1); + } else { + break; + } + } + } + + fields.add(new RecordField(GrokRecordReader.STACK_TRACE_COLUMN_NAME, RecordFieldType.STRING.getDataType())); + + final RecordSchema schema = new SimpleRecordSchema(fields); + return schema; + } + + @Override - protected boolean isSchemaRequired() { - return false; + protected List getSchemaAccessStrategyValues() { + final List allowableValues = new ArrayList<>(); + allowableValues.add(STRING_FIELDS_FROM_GROK_EXPRESSION); + allowableValues.addAll(super.getSchemaAccessStrategyValues()); + return allowableValues; + } + + @Override + protected AllowableValue getDefaultSchemaAccessStrategy() { + return STRING_FIELDS_FROM_GROK_EXPRESSION; + } + + @Override + protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) { + if (allowableValue.equalsIgnoreCase(STRING_FIELDS_FROM_GROK_EXPRESSION.getValue())) { + return new SchemaAccessStrategy() { + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { + return recordSchema; + } + }; + } else { + return super.getSchemaAccessStrategy(allowableValue, schemaRegistry); + } } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException { - final RecordSchema schema = useSchemaRegistry ? getSchema(flowFile) : null; - return new GrokRecordReader(in, grok, schema); + public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { + final RecordSchema schema = getSchema(flowFile, in); + return new GrokRecordReader(in, grok, schema, appendUnmatchedLine); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java index 458dbd8a31d8..c00866535e61 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java @@ -21,40 +21,33 @@ import java.io.IOException; import java.io.InputStream; import java.io.InputStreamReader; -import java.text.ParseException; -import java.util.ArrayList; import java.util.Collections; -import java.util.Date; import java.util.HashMap; import java.util.List; import java.util.Map; -import java.util.TimeZone; -import java.util.regex.Matcher; import java.util.regex.Pattern; -import org.apache.commons.lang3.time.FastDateFormat; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; 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.util.DataTypeUtils; import io.thekraken.grok.api.Grok; -import io.thekraken.grok.api.GrokUtils; import io.thekraken.grok.api.Match; public class GrokRecordReader implements RecordReader { private final BufferedReader reader; private final Grok grok; + private final boolean append; private RecordSchema schema; private String nextLine; - static final String STACK_TRACE_COLUMN_NAME = "STACK_TRACE"; + static final String STACK_TRACE_COLUMN_NAME = "stackTrace"; private static final Pattern STACK_TRACE_PATTERN = Pattern.compile( "^\\s*(?:(?: |\\t)+at )|" + "(?:(?: |\\t)+\\[CIRCULAR REFERENCE\\:)|" @@ -62,21 +55,11 @@ public class GrokRecordReader implements RecordReader { + "(?:Suppressed\\: )|" + "(?:\\s+... \\d+ (?:more|common frames? omitted)$)"); - private static final FastDateFormat TIME_FORMAT_DATE; - private static final FastDateFormat TIME_FORMAT_TIME; - private static final FastDateFormat TIME_FORMAT_TIMESTAMP; - - static { - final TimeZone gmt = TimeZone.getTimeZone("GMT"); - TIME_FORMAT_DATE = FastDateFormat.getInstance("yyyy-MM-dd", gmt); - TIME_FORMAT_TIME = FastDateFormat.getInstance("HH:mm:ss", gmt); - TIME_FORMAT_TIMESTAMP = FastDateFormat.getInstance("yyyy-MM-dd HH:mm:ss", gmt); - } - - public GrokRecordReader(final InputStream in, final Grok grok, final RecordSchema schema) { + public GrokRecordReader(final InputStream in, final Grok grok, final RecordSchema schema, final boolean append) { this.reader = new BufferedReader(new InputStreamReader(in)); this.grok = grok; this.schema = schema; + this.append = append; } @Override @@ -115,7 +98,7 @@ public Record nextRecord() throws IOException, MalformedRecordException { if (isStartOfStackTrace(nextLine)) { stackTrace = readStackTrace(nextLine); break; - } else { + } else if (append) { toAppend.append("\n").append(nextLine); } } else { @@ -136,12 +119,12 @@ public Record nextRecord() throws IOException, MalformedRecordException { } final DataType fieldType = schema.getDataType(fieldName).orElse(null); - final Object converted = convert(fieldType, value.toString()); + final Object converted = convert(fieldType, value.toString(), fieldName); values.put(fieldName, converted); } - final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2); - if (toAppend.length() > 0) { + if (append && toAppend.length() > 0) { + final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2); final Object existingValue = values.get(lastFieldBeforeStackTrace); final String updatedValue = existingValue == null ? toAppend.toString() : existingValue + toAppend.toString(); values.put(lastFieldBeforeStackTrace, updatedValue); @@ -205,7 +188,7 @@ private boolean isLineInStackTrace(final String line) { } - protected Object convert(final DataType fieldType, final String string) { + protected Object convert(final DataType fieldType, final String string, final String fieldName) { if (fieldType == null) { return string; } @@ -220,79 +203,12 @@ protected Object convert(final DataType fieldType, final String string) { return null; } - switch (fieldType.getFieldType()) { - case BOOLEAN: - return Boolean.parseBoolean(string); - case BYTE: - return Byte.parseByte(string); - case SHORT: - return Short.parseShort(string); - case INT: - return Integer.parseInt(string); - case LONG: - return Long.parseLong(string); - case FLOAT: - return Float.parseFloat(string); - case DOUBLE: - return Double.parseDouble(string); - case DATE: - try { - Date date = TIME_FORMAT_DATE.parse(string); - return new java.sql.Date(date.getTime()); - } catch (ParseException e) { - return null; - } - case TIME: - try { - Date date = TIME_FORMAT_TIME.parse(string); - return new java.sql.Time(date.getTime()); - } catch (ParseException e) { - return null; - } - case TIMESTAMP: - try { - Date date = TIME_FORMAT_TIMESTAMP.parse(string); - return new java.sql.Timestamp(date.getTime()); - } catch (ParseException e) { - return null; - } - case STRING: - default: - return string; - } + return DataTypeUtils.convertType(string, fieldType, fieldName); } @Override public RecordSchema getSchema() { - if (schema != null) { - return schema; - } - - final List fields = new ArrayList<>(); - - String grokExpression = grok.getOriginalGrokPattern(); - while (grokExpression.length() > 0) { - final Matcher matcher = GrokUtils.GROK_PATTERN.matcher(grokExpression); - if (matcher.find()) { - final Map namedGroups = GrokUtils.namedGroups(matcher, grokExpression); - final String fieldName = namedGroups.get("subname"); - - DataType dataType = RecordFieldType.STRING.getDataType(); - final RecordField recordField = new RecordField(fieldName, dataType); - fields.add(recordField); - - if (grokExpression.length() > matcher.end() + 1) { - grokExpression = grokExpression.substring(matcher.end() + 1); - } else { - break; - } - } - } - - fields.add(new RecordField(STACK_TRACE_COLUMN_NAME, RecordFieldType.STRING.getDataType())); - - schema = new SimpleRecordSchema(fields); return schema; } 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/AbstractJsonRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java index ad049121a635..c5c5fb09c33c 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/AbstractJsonRowRecordReader.java @@ -19,20 +19,12 @@ import java.io.IOException; import java.io.InputStream; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.List; -import java.util.Map; import java.util.Optional; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.SimpleRecordSchema; -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 org.apache.nifi.serialization.record.RecordSchema; import org.codehaus.jackson.JsonFactory; import org.codehaus.jackson.JsonNode; @@ -98,55 +90,6 @@ public Record nextRecord() throws IOException, MalformedRecordException { } } - protected DataType determineFieldType(final JsonNode node) { - if (node.isDouble()) { - return RecordFieldType.DOUBLE.getDataType(); - } - if (node.isBoolean()) { - return RecordFieldType.BOOLEAN.getDataType(); - } - if (node.isFloatingPointNumber()) { - return RecordFieldType.FLOAT.getDataType(); - } - if (node.isBigInteger()) { - return RecordFieldType.BIGINT.getDataType(); - } - if (node.isBigDecimal()) { - return RecordFieldType.DOUBLE.getDataType(); - } - if (node.isLong()) { - return RecordFieldType.LONG.getDataType(); - } - if (node.isInt()) { - return RecordFieldType.INT.getDataType(); - } - if (node.isTextual()) { - return RecordFieldType.STRING.getDataType(); - } - if (node.isArray()) { - return RecordFieldType.ARRAY.getDataType(); - } - - final RecordSchema childSchema = determineSchema(node); - return RecordFieldType.RECORD.getRecordDataType(childSchema); - } - - protected RecordSchema determineSchema(final JsonNode jsonNode) { - final List recordFields = new ArrayList<>(); - - final Iterator> itr = jsonNode.getFields(); - while (itr.hasNext()) { - final Map.Entry entry = itr.next(); - final String elementName = entry.getKey(); - final JsonNode node = entry.getValue(); - - DataType dataType = determineFieldType(node); - recordFields.add(new RecordField(elementName, dataType)); - } - - return new SimpleRecordSchema(recordFields); - } - protected Object getRawNodeValue(final JsonNode fieldNode) throws IOException { if (fieldNode == null || !fieldNode.isValueNode()) { return null; 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/JsonPathReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java index 467ecf85e748..2d11a9b6f955 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathReader.java @@ -36,11 +36,12 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.DateTimeUtils; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; -import org.apache.nifi.serialization.SchemaRegistryRecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaRegistryService; import org.apache.nifi.serialization.record.RecordSchema; import com.jayway.jsonpath.JsonPath; @@ -56,7 +57,7 @@ + "field whose name is the same as the property name.", description="User-defined properties identifiy how to extract specific fields from a JSON object in order to create a Record", supportsExpressionLanguage=false) -public class JsonPathReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory { +public class JsonPathReader extends SchemaRegistryService implements RecordReaderFactory { private volatile String dateFormat; private volatile String timeFormat; @@ -127,8 +128,8 @@ protected Collection customValidate(final ValidationContext va } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException { - final RecordSchema schema = getSchema(flowFile); + public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + final RecordSchema schema = getSchema(flowFile, in); return new JsonPathRowRecordReader(jsonPaths, schema, in, logger, dateFormat, timeFormat, timestampFormat); } 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/JsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java index a0f3c322f6f6..28340452694f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java @@ -106,7 +106,7 @@ protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSc value = null; } - value = convert(value, desiredType); + value = convert(value, desiredType, fieldName); values.put(fieldName, value); } @@ -115,7 +115,7 @@ protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSc @SuppressWarnings("unchecked") - protected Object convert(final Object value, final DataType dataType) { + protected Object convert(final Object value, final DataType dataType, final String fieldName) { if (value == null) { return null; } @@ -131,7 +131,7 @@ protected Object convert(final Object value, final DataType dataType) { final Object[] coercedValues = new Object[list.size()]; int i = 0; for (final Object rawValue : list) { - coercedValues[i++] = DataTypeUtils.convertType(rawValue, arrayType.getElementType(), dateFormat, timeFormat, timestampFormat); + coercedValues[i++] = convert(rawValue, arrayType.getElementType(), fieldName); } return coercedValues; } @@ -147,14 +147,14 @@ protected Object convert(final Object value, final DataType dataType) { final String key = entry.getKey(); final Optional desiredTypeOption = childSchema.getDataType(key); if (desiredTypeOption.isPresent()) { - final Object coercedValue = DataTypeUtils.convertType(entry.getValue(), desiredTypeOption.get(), dateFormat, timeFormat, timestampFormat); + final Object coercedValue = convert(entry.getValue(), desiredTypeOption.get(), fieldName + "." + key); coercedValues.put(key, coercedValue); } } return new MapRecord(childSchema, coercedValues); } else { - return DataTypeUtils.convertType(value, dataType, dateFormat, timeFormat, timestampFormat); + return DataTypeUtils.convertType(value, dataType, dateFormat, timeFormat, timestampFormat, fieldName); } } 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 d09f13547b20..9e5454724aec 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 @@ -17,6 +17,8 @@ package org.apache.nifi.json; +import java.io.IOException; +import java.io.InputStream; import java.util.ArrayList; import java.util.List; @@ -25,10 +27,13 @@ import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.DateTimeTextRecordSetWriter; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; +import org.apache.nifi.serialization.record.RecordSchema; @Tags({"json", "resultset", "writer", "serialize", "record", "recordset", "row"}) @CapabilityDescription("Writes the results of a RecordSet as a JSON Array. Even if the RecordSet " @@ -59,8 +64,9 @@ public void onEnabled(final ConfigurationContext context) { } @Override - public RecordSetWriter createWriter(final ComponentLog logger) { - return new WriteJsonResult(logger, prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat()); + public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream flowFileContent) throws SchemaNotFoundException, IOException { + final RecordSchema schema = getSchema(flowFile, flowFileContent); + return new WriteJsonResult(logger, schema, getSchemaAccessWriter(), prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat()); } } 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/JsonTreeReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java index 1abb1f451e93..1dd983411052 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeReader.java @@ -30,11 +30,12 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.DateTimeUtils; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; -import org.apache.nifi.serialization.RowRecordReaderFactory; -import org.apache.nifi.serialization.SchemaRegistryRecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; +import org.apache.nifi.serialization.SchemaRegistryService; @Tags({"json", "tree", "record", "reader", "parser"}) @CapabilityDescription("Parses JSON into individual Record objects. The Record that is produced will contain all top-level " @@ -45,7 +46,7 @@ + "a field that is not present in the schema, that field will be skipped. " + "See the Usage of the Controller Service for more information and examples.") @SeeAlso(JsonPathReader.class) -public class JsonTreeReader extends SchemaRegistryRecordReader implements RowRecordReaderFactory { +public class JsonTreeReader extends SchemaRegistryService implements RecordReaderFactory { private volatile String dateFormat; private volatile String timeFormat; @@ -68,7 +69,7 @@ public void storeFormats(final ConfigurationContext context) { } @Override - public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException { - return new JsonTreeRowRecordReader(in, logger, getSchema(flowFile), dateFormat, timeFormat, timestampFormat); + public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, MalformedRecordException, SchemaNotFoundException { + return new JsonTreeRowRecordReader(in, logger, getSchema(flowFile, in), dateFormat, timeFormat, timestampFormat); } } 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/JsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java index c8d07f4bf6d2..e396e8eea3fe 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java @@ -19,17 +19,23 @@ import java.io.IOException; import java.io.InputStream; +import java.util.ArrayList; import java.util.HashMap; +import java.util.Iterator; +import java.util.List; import java.util.Map; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.serialization.MalformedRecordException; +import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; 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.type.ArrayDataType; +import org.apache.nifi.serialization.record.type.MapDataType; import org.apache.nifi.serialization.record.type.RecordDataType; import org.apache.nifi.serialization.record.util.DataTypeUtils; import org.codehaus.jackson.JsonNode; @@ -55,6 +61,10 @@ public JsonTreeRowRecordReader(final InputStream in, final ComponentLog logger, @Override protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema) throws IOException, MalformedRecordException { + return convertJsonNodeToRecord(jsonNode, schema, ""); + } + + private Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSchema schema, final String fieldNamePrefix) throws IOException, MalformedRecordException { if (jsonNode == null) { return null; } @@ -66,7 +76,7 @@ protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSc final JsonNode fieldNode = jsonNode.get(fieldName); final DataType desiredType = field.getDataType(); - final Object value = convertField(fieldNode, fieldName, desiredType); + final Object value = convertField(fieldNode, fieldNamePrefix + fieldName, desiredType); values.put(fieldName, value); } @@ -80,42 +90,50 @@ protected Object convertField(final JsonNode fieldNode, final String fieldName, switch (desiredType.getFieldType()) { case BOOLEAN: - return DataTypeUtils.toBoolean(getRawNodeValue(fieldNode)); + return DataTypeUtils.toBoolean(getRawNodeValue(fieldNode), fieldName); case BYTE: - return DataTypeUtils.toByte(getRawNodeValue(fieldNode)); + return DataTypeUtils.toByte(getRawNodeValue(fieldNode), fieldName); case CHAR: - return DataTypeUtils.toCharacter(getRawNodeValue(fieldNode)); + return DataTypeUtils.toCharacter(getRawNodeValue(fieldNode), fieldName); case DOUBLE: - return DataTypeUtils.toDouble(getRawNodeValue(fieldNode)); + return DataTypeUtils.toDouble(getRawNodeValue(fieldNode), fieldName); case FLOAT: - return DataTypeUtils.toFloat(getRawNodeValue(fieldNode)); + return DataTypeUtils.toFloat(getRawNodeValue(fieldNode), fieldName); case INT: - return DataTypeUtils.toInteger(getRawNodeValue(fieldNode)); + return DataTypeUtils.toInteger(getRawNodeValue(fieldNode), fieldName); case LONG: - return DataTypeUtils.toLong(getRawNodeValue(fieldNode)); + return DataTypeUtils.toLong(getRawNodeValue(fieldNode), fieldName); case SHORT: - return DataTypeUtils.toShort(getRawNodeValue(fieldNode)); + return DataTypeUtils.toShort(getRawNodeValue(fieldNode), fieldName); case STRING: return DataTypeUtils.toString(getRawNodeValue(fieldNode), dateFormat, timeFormat, timestampFormat); case DATE: - return DataTypeUtils.toDate(getRawNodeValue(fieldNode), dateFormat); + return DataTypeUtils.toDate(getRawNodeValue(fieldNode), dateFormat, fieldName); case TIME: - return DataTypeUtils.toTime(getRawNodeValue(fieldNode), timeFormat); + return DataTypeUtils.toTime(getRawNodeValue(fieldNode), timeFormat, fieldName); case TIMESTAMP: - return DataTypeUtils.toTimestamp(getRawNodeValue(fieldNode), timestampFormat); + return DataTypeUtils.toTimestamp(getRawNodeValue(fieldNode), timestampFormat, fieldName); + case MAP: { + final DataType valueType = ((MapDataType) desiredType).getValueType(); + + final Map map = new HashMap<>(); + final Iterator fieldNameItr = fieldNode.getFieldNames(); + while (fieldNameItr.hasNext()) { + final String childName = fieldNameItr.next(); + final JsonNode childNode = fieldNode.get(childName); + final Object childValue = convertField(childNode, fieldName + "." + childName, valueType); + map.put(childName, childValue); + } + + return map; + } case ARRAY: { final ArrayNode arrayNode = (ArrayNode) fieldNode; final int numElements = arrayNode.size(); final Object[] arrayElements = new Object[numElements]; int count = 0; for (final JsonNode node : arrayNode) { - final DataType elementType; - if (desiredType instanceof ArrayDataType) { - elementType = ((ArrayDataType) desiredType).getElementType(); - } else { - elementType = determineFieldType(node); - } - + final DataType elementType = ((ArrayDataType) desiredType).getElementType(); final Object converted = convertField(node, fieldName, elementType); arrayElements[count++] = converted; } @@ -124,14 +142,24 @@ protected Object convertField(final JsonNode fieldNode, final String fieldName, } case RECORD: { if (fieldNode.isObject()) { - final RecordSchema childSchema; + RecordSchema childSchema; if (desiredType instanceof RecordDataType) { childSchema = ((RecordDataType) desiredType).getChildSchema(); } else { return null; } - return convertJsonNodeToRecord(fieldNode, childSchema); + if (childSchema == null) { + final List fields = new ArrayList<>(); + final Iterator fieldNameItr = fieldNode.getFieldNames(); + while (fieldNameItr.hasNext()) { + fields.add(new RecordField(fieldNameItr.next(), RecordFieldType.STRING.getDataType())); + } + + childSchema = new SimpleRecordSchema(fields); + } + + return convertJsonNodeToRecord(fieldNode, childSchema, fieldName + "."); } else { return null; } 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/WriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java index 05895d83f4ab..3c9517a55d6b 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java @@ -22,8 +22,10 @@ import java.math.BigInteger; import java.sql.SQLException; import java.util.Collections; +import java.util.Map; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaAccessWriter; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.DataType; @@ -33,6 +35,8 @@ import org.apache.nifi.serialization.record.RecordSet; import org.apache.nifi.serialization.record.type.ArrayDataType; import org.apache.nifi.serialization.record.type.ChoiceDataType; +import org.apache.nifi.serialization.record.type.MapDataType; +import org.apache.nifi.serialization.record.type.RecordDataType; import org.apache.nifi.serialization.record.util.DataTypeUtils; import org.apache.nifi.stream.io.NonCloseableOutputStream; import org.codehaus.jackson.JsonFactory; @@ -42,25 +46,32 @@ public class WriteJsonResult implements RecordSetWriter { private final ComponentLog logger; private final boolean prettyPrint; + private final SchemaAccessWriter schemaAccess; + private final RecordSchema recordSchema; private final JsonFactory factory = new JsonFactory(); private final String dateFormat; private final String timeFormat; private final String timestampFormat; - public WriteJsonResult(final ComponentLog logger, final boolean prettyPrint, final String dateFormat, final String timeFormat, final String timestampFormat) { + public WriteJsonResult(final ComponentLog logger, final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final boolean prettyPrint, + final String dateFormat, final String timeFormat, final String timestampFormat) { + + this.logger = logger; + this.recordSchema = recordSchema; this.prettyPrint = prettyPrint; + this.schemaAccess = schemaAccess; this.dateFormat = dateFormat; this.timeFormat = timeFormat; this.timestampFormat = timestampFormat; - - this.logger = logger; } @Override public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws IOException { int count = 0; + schemaAccess.writeHeader(recordSchema, rawOut); + try (final JsonGenerator generator = factory.createJsonGenerator(new NonCloseableOutputStream(rawOut))) { if (prettyPrint) { generator.useDefaultPrettyPrinter(); @@ -71,7 +82,7 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I Record record; while ((record = rs.next()) != null) { count++; - writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject()); + writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject()); } generator.writeEndArray(); @@ -79,7 +90,7 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I throw new IOException("Failed to serialize Result Set to stream", e); } - return WriteResult.of(count, Collections.emptyMap()); + return WriteResult.of(count, schemaAccess.getAttributes(recordSchema)); } @Override @@ -89,7 +100,7 @@ public WriteResult write(final Record record, final OutputStream rawOut) throws generator.useDefaultPrettyPrinter(); } - writeRecord(record, generator, g -> g.writeStartObject(), g -> g.writeEndObject()); + writeRecord(record, recordSchema, generator, g -> g.writeStartObject(), g -> g.writeEndObject()); } catch (final SQLException e) { throw new IOException("Failed to write records to stream", e); } @@ -97,14 +108,13 @@ public WriteResult write(final Record record, final OutputStream rawOut) throws return WriteResult.of(1, Collections.emptyMap()); } - private void writeRecord(final Record record, final JsonGenerator generator, final GeneratorTask startTask, final GeneratorTask endTask) + private void writeRecord(final Record record, final RecordSchema writeSchema, final JsonGenerator generator, final GeneratorTask startTask, final GeneratorTask endTask) throws JsonGenerationException, IOException, SQLException { try { - final RecordSchema schema = record.getSchema(); startTask.apply(generator); - for (int i = 0; i < schema.getFieldCount(); i++) { - final String fieldName = schema.getField(i).getFieldName(); + for (int i = 0; i < writeSchema.getFieldCount(); i++) { + final String fieldName = writeSchema.getField(i).getFieldName(); final Object value = record.getValue(fieldName); if (value == null) { generator.writeNullField(fieldName); @@ -112,9 +122,9 @@ private void writeRecord(final Record record, final JsonGenerator generator, fin } generator.writeFieldName(fieldName); - final DataType dataType = schema.getDataType(fieldName).get(); + final DataType dataType = writeSchema.getDataType(fieldName).get(); - writeValue(generator, value, dataType, i < schema.getFieldCount() - 1); + writeValue(generator, value, fieldName, dataType, i < writeSchema.getFieldCount() - 1); } endTask.apply(generator); @@ -125,7 +135,8 @@ private void writeRecord(final Record record, final JsonGenerator generator, fin } - private void writeValue(final JsonGenerator generator, final Object value, final DataType dataType, final boolean moreCols) + @SuppressWarnings("unchecked") + private void writeValue(final JsonGenerator generator, final Object value, final String fieldName, final DataType dataType, final boolean moreCols) throws JsonGenerationException, IOException, SQLException { if (value == null) { generator.writeNull(); @@ -133,7 +144,7 @@ private void writeValue(final JsonGenerator generator, final Object value, final } final DataType chosenDataType = dataType.getFieldType() == RecordFieldType.CHOICE ? DataTypeUtils.chooseDataType(value, (ChoiceDataType) dataType) : dataType; - final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType); + final Object coercedValue = DataTypeUtils.convertType(value, chosenDataType, fieldName); if (coercedValue == null) { generator.writeNull(); return; @@ -146,18 +157,18 @@ private void writeValue(final JsonGenerator generator, final Object value, final generator.writeString(DataTypeUtils.toString(coercedValue, dateFormat, timeFormat, timestampFormat)); break; case DOUBLE: - generator.writeNumber(DataTypeUtils.toDouble(coercedValue)); + generator.writeNumber(DataTypeUtils.toDouble(coercedValue, fieldName)); break; case FLOAT: - generator.writeNumber(DataTypeUtils.toFloat(coercedValue)); + generator.writeNumber(DataTypeUtils.toFloat(coercedValue, fieldName)); break; case LONG: - generator.writeNumber(DataTypeUtils.toLong(coercedValue)); + generator.writeNumber(DataTypeUtils.toLong(coercedValue, fieldName)); break; case INT: case BYTE: case SHORT: - generator.writeNumber(DataTypeUtils.toInteger(coercedValue)); + generator.writeNumber(DataTypeUtils.toInteger(coercedValue, fieldName)); break; case CHAR: case STRING: @@ -182,7 +193,24 @@ private void writeValue(final JsonGenerator generator, final Object value, final break; case RECORD: { final Record record = (Record) coercedValue; - writeRecord(record, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject()); + final RecordDataType recordDataType = (RecordDataType) chosenDataType; + final RecordSchema childSchema = recordDataType.getChildSchema(); + writeRecord(record, childSchema, generator, gen -> gen.writeStartObject(), gen -> gen.writeEndObject()); + break; + } + case MAP: { + final MapDataType mapDataType = (MapDataType) chosenDataType; + final DataType valueDataType = mapDataType.getValueType(); + final Map map = (Map) coercedValue; + generator.writeStartObject(); + int i = 0; + for (final Map.Entry entry : map.entrySet()) { + final String mapKey = entry.getKey(); + final Object mapValue = entry.getValue(); + generator.writeFieldName(mapKey); + writeValue(generator, mapValue, fieldName + "." + mapKey, valueDataType, ++i < map.size()); + } + generator.writeEndObject(); break; } case ARRAY: @@ -191,7 +219,7 @@ private void writeValue(final JsonGenerator generator, final Object value, final final Object[] values = (Object[]) coercedValue; final ArrayDataType arrayDataType = (ArrayDataType) dataType; final DataType elementType = arrayDataType.getElementType(); - writeArray(values, generator, elementType); + writeArray(values, fieldName, generator, elementType); } else { generator.writeString(coercedValue.toString()); } @@ -199,12 +227,13 @@ private void writeValue(final JsonGenerator generator, final Object value, final } } - private void writeArray(final Object[] values, final JsonGenerator generator, final DataType elementType) throws JsonGenerationException, IOException, SQLException { + private void writeArray(final Object[] values, final String fieldName, final JsonGenerator generator, final DataType elementType) + throws JsonGenerationException, IOException, SQLException { generator.writeStartArray(); for (int i = 0; i < values.length; i++) { final boolean moreEntries = i < values.length - 1; final Object element = values[i]; - writeValue(generator, element, elementType, moreEntries); + writeValue(generator, element, fieldName, elementType, moreEntries); } generator.writeEndArray(); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java new file mode 100644 index 000000000000..2b8d9ba74a67 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.schema.access; + +import java.io.InputStream; + +import org.apache.avro.Schema; +import org.apache.nifi.avro.AvroTypeUtil; +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.serialization.record.RecordSchema; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class AvroSchemaTextStrategy implements SchemaAccessStrategy { + private static final Logger logger = LoggerFactory.getLogger(AvroSchemaTextStrategy.class); + private final PropertyValue schemaTextPropertyValue; + + public AvroSchemaTextStrategy(final PropertyValue schemaTextPropertyValue) { + this.schemaTextPropertyValue = schemaTextPropertyValue; + } + + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { + final String schemaText = schemaTextPropertyValue.evaluateAttributeExpressions(flowFile).getValue(); + if (schemaText == null) { + throw new SchemaNotFoundException("FlowFile did not contain appropriate attributes to determine Schema Text"); + } + + logger.debug("For {} found schema text {}", flowFile, schemaText); + + try { + final Schema avroSchema = new Schema.Parser().parse(schemaText); + return AvroTypeUtil.createSchema(avroSchema); + } catch (final Exception e) { + throw new SchemaNotFoundException("Failed to create schema from the Schema Text after evaluating FlowFile Attributes", e); + } + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java new file mode 100644 index 000000000000..3ce2cb736f17 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java @@ -0,0 +1,60 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.stream.io.StreamUtils; + +public class HortonworksEncodedSchemaReferenceStrategy implements SchemaAccessStrategy { + private static final int LATEST_PROTOCOL_VERSION = 1; + private final SchemaRegistry schemaRegistry; + + public HortonworksEncodedSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) { + this.schemaRegistry = schemaRegistry; + } + + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException, IOException { + final byte[] buffer = new byte[13]; + try { + StreamUtils.fillBuffer(contentStream, buffer); + } catch (final IOException ioe) { + throw new SchemaNotFoundException("Could not read first 13 bytes from stream", ioe); + } + + final ByteBuffer bb = ByteBuffer.wrap(buffer); + final int protocolVersion = bb.get(); + if (protocolVersion != 1) { + throw new SchemaNotFoundException("Schema Encoding appears to be of an incompatible version. The latest known Protocol is Version " + + LATEST_PROTOCOL_VERSION + " but the data was encoded with version " + protocolVersion); + } + + final long schemaId = bb.getLong(); + final int schemaVersion = bb.getInt(); + + return schemaRegistry.retrieveSchema(schemaId, schemaVersion); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java new file mode 100644 index 000000000000..f7019e719572 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java @@ -0,0 +1,60 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.Map; +import java.util.OptionalInt; +import java.util.OptionalLong; + +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; + +public class HortonworksEncodedSchemaReferenceWriter implements SchemaAccessWriter { + private static final int LATEST_PROTOCOL_VERSION = 1; + + @Override + public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException { + final SchemaIdentifier identifier = schema.getIdentifier(); + final OptionalLong identifierOption = identifier.getIdentifier(); + if (!identifierOption.isPresent()) { + throw new IOException("Cannot write Encoded Schema Reference for Schema because the Schema Identifier is not known"); + } + + final OptionalInt versionOption = identifier.getVersion(); + if (!versionOption.isPresent()) { + throw new IOException("Cannot write Encoded Schema Reference for Schema because the Schema Version is not known"); + } + + final ByteBuffer bb = ByteBuffer.allocate(13); + bb.put((byte) LATEST_PROTOCOL_VERSION); + bb.putLong(identifierOption.getAsLong()); + bb.putInt(versionOption.getAsInt()); + + out.write(bb.array()); + } + + @Override + public Map getAttributes(final RecordSchema schema) { + return Collections.emptyMap(); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java new file mode 100644 index 000000000000..d8dd0a9ec8d2 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java @@ -0,0 +1,29 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.serialization.record.RecordSchema; + +public interface SchemaAccessStrategy { + RecordSchema getSchema(FlowFile flowFile, InputStream contentStream, ConfigurationContext context) throws SchemaNotFoundException, IOException; +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java new file mode 100644 index 000000000000..4a5d76394ad4 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java @@ -0,0 +1,47 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Map; + +import org.apache.nifi.serialization.record.RecordSchema; + +public interface SchemaAccessWriter { + + /** + * Writes the given Record Schema to the given OutputStream as header information, if appropriate, + * or returns without writing anything if the implementation does not need to write information to + * the contents of the FlowFile + * + * @param schema the schema to write + * @param out the OutputStream to write to + * @throws IOException if unable to write to the given stream + */ + void writeHeader(RecordSchema schema, OutputStream out) throws IOException; + + /** + * Returns a Map of String to String that represent the attributes that should be added to the FlowFile, or + * an empty map if no attributes should be added. + * + * @return a Map of attributes to add to the FlowFile. + */ + Map getAttributes(RecordSchema schema); + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java new file mode 100644 index 000000000000..c4bb86364905 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java @@ -0,0 +1,46 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; + +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; + +public class SchemaNameAsAttribute implements SchemaAccessWriter { + private static final String SCHEMA_NAME_ATTRIBUTE = "schema.name"; + + @Override + public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException { + } + + @Override + public Map getAttributes(final RecordSchema schema) { + final SchemaIdentifier identifier = schema.getIdentifier(); + final Optional nameOption = identifier.getName(); + if (nameOption.isPresent()) { + return Collections.singletonMap(SCHEMA_NAME_ATTRIBUTE, nameOption.get()); + } + return Collections.emptyMap(); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java new file mode 100644 index 000000000000..b39a1176e1df --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.schema.access; + +import java.io.InputStream; + +import org.apache.nifi.components.PropertyValue; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; +import org.apache.nifi.serialization.record.RecordSchema; + +public class SchemaNamePropertyStrategy implements SchemaAccessStrategy { + private final SchemaRegistry schemaRegistry; + private final PropertyValue schemaNamePropertyValue; + + public SchemaNamePropertyStrategy(final SchemaRegistry schemaRegistry, final PropertyValue schemaNamePropertyValue) { + this.schemaRegistry = schemaRegistry; + this.schemaNamePropertyValue = schemaNamePropertyValue; + } + + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { + final String schemaName = schemaNamePropertyValue.evaluateAttributeExpressions(flowFile).getValue(); + if (schemaName.trim().isEmpty()) { + throw new SchemaNotFoundException("FlowFile did not contain appropriate attributes to determine Schema Name."); + } + + try { + final RecordSchema recordSchema = schemaRegistry.retrieveSchema(schemaName); + if (recordSchema == null) { + throw new SchemaNotFoundException("Could not find a schema with name '" + schemaName + "' in the configured Schema Registry"); + } + + return recordSchema; + } catch (final Exception e) { + throw new SchemaNotFoundException("Could not retrieve schema with name '" + schemaName + "' from the configured Schema Registry", e); + } + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java new file mode 100644 index 000000000000..ba98fb4524bd --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java @@ -0,0 +1,46 @@ +/* + * 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.schema.access; + +import java.io.OutputStream; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; + +import org.apache.nifi.serialization.record.RecordSchema; + +public class SchemaTextAsAttribute implements SchemaAccessWriter { + + @Override + public void writeHeader(final RecordSchema schema, final OutputStream out) { + } + + @Override + public Map getAttributes(final RecordSchema schema) { + final Optional textFormatOption = schema.getSchemaFormat(); + if (textFormatOption.isPresent()) { + final Optional textOption = schema.getSchemaText(); + if (textOption.isPresent()) { + return Collections.singletonMap(textFormatOption.get() + ".schema", textOption.get()); + } + } + + return Collections.emptyMap(); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java index 55450901cc2a..2260c2e82963 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/DateTimeTextRecordSetWriter.java @@ -17,15 +17,14 @@ package org.apache.nifi.serialization; -import java.util.Arrays; +import java.util.ArrayList; import java.util.List; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; -public abstract class DateTimeTextRecordSetWriter extends AbstractControllerService { +public abstract class DateTimeTextRecordSetWriter extends SchemaRegistryRecordSetWriter { private volatile String dateFormat; private volatile String timeFormat; @@ -33,7 +32,11 @@ public abstract class DateTimeTextRecordSetWriter extends AbstractControllerServ @Override protected List getSupportedPropertyDescriptors() { - return Arrays.asList(DateTimeUtils.DATE_FORMAT, DateTimeUtils.TIME_FORMAT, DateTimeUtils.TIMESTAMP_FORMAT); + final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + properties.add(DateTimeUtils.DATE_FORMAT); + properties.add(DateTimeUtils.TIME_FORMAT); + properties.add(DateTimeUtils.TIMESTAMP_FORMAT); + return properties; } @OnEnabled diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java deleted file mode 100644 index ee25e648e6fa..000000000000 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordReader.java +++ /dev/null @@ -1,110 +0,0 @@ -/* - * 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; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; - -import org.apache.nifi.annotation.lifecycle.OnEnabled; -import org.apache.nifi.components.PropertyDescriptor; -import org.apache.nifi.components.PropertyValue; -import org.apache.nifi.components.ValidationContext; -import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.controller.AbstractControllerService; -import org.apache.nifi.controller.ConfigurationContext; -import org.apache.nifi.flowfile.FlowFile; -import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.schemaregistry.services.SchemaRegistry; -import org.apache.nifi.serialization.record.RecordSchema; - -public abstract class SchemaRegistryRecordReader extends AbstractControllerService { - - protected static final PropertyDescriptor REQUIRED_SCHEMA_REGISTRY = new PropertyDescriptor.Builder() - .name("Schema Registry") - .description("Specifies the Controller Service to use for the Schema Registry") - .identifiesControllerService(SchemaRegistry.class) - .required(true) - .build(); - - protected static final PropertyDescriptor OPTIONAL_SCHEMA_REGISTRY = new PropertyDescriptor.Builder() - .fromPropertyDescriptor(REQUIRED_SCHEMA_REGISTRY) - .required(false) - .build(); - - protected static final PropertyDescriptor REQUIRED_SCHEMA_NAME = new PropertyDescriptor.Builder() - .name("Schema Name") - .description("Name of the Schema that is stored in the Schema Registry") - .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) - .expressionLanguageSupported(true) - .required(true) - .build(); - - protected static final PropertyDescriptor OPTIONAL_SCHEMA_NAME = new PropertyDescriptor.Builder() - .fromPropertyDescriptor(REQUIRED_SCHEMA_NAME) - .required(false) - .build(); - - - private volatile SchemaRegistry schemaRegistry; - private volatile PropertyValue schemaName; - - @Override - protected List getSupportedPropertyDescriptors() { - final List properties = new ArrayList<>(2); - if (isSchemaRequired()) { - properties.add(REQUIRED_SCHEMA_REGISTRY); - properties.add(REQUIRED_SCHEMA_NAME); - } else { - properties.add(OPTIONAL_SCHEMA_REGISTRY); - properties.add(OPTIONAL_SCHEMA_NAME); - } - - return properties; - } - - @OnEnabled - public void storeRegistryValues(final ConfigurationContext context) { - schemaRegistry = context.getProperty(REQUIRED_SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class); - schemaName = context.getProperty(REQUIRED_SCHEMA_NAME); - } - - public RecordSchema getSchema(final FlowFile flowFile) { - final String evaluatedSchemaName = schemaName.evaluateAttributeExpressions(flowFile).getValue(); - final RecordSchema schema = schemaRegistry.retrieveSchema(evaluatedSchemaName); - return schema; - } - - @Override - protected Collection customValidate(final ValidationContext validationContext) { - if (validationContext.getProperty(OPTIONAL_SCHEMA_REGISTRY).isSet() && !validationContext.getProperty(OPTIONAL_SCHEMA_NAME).isSet()) { - return Collections.singleton(new ValidationResult.Builder() - .subject("Schema Registry") - .explanation("If the Schema Registry is configured, the Schema name must also be configured") - .valid(false) - .build()); - } - - return Collections.emptyList(); - } - - protected boolean isSchemaRequired() { - return true; - } -} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java new file mode 100644 index 000000000000..cd57e8b49c4a --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java @@ -0,0 +1,113 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceWriter; +import org.apache.nifi.schema.access.SchemaAccessWriter; +import org.apache.nifi.schema.access.SchemaNameAsAttribute; +import org.apache.nifi.schema.access.SchemaTextAsAttribute; + +public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService { + + static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Use '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", "Use '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", + "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. " + + "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."); + + protected static final PropertyDescriptor SCHEMA_WRITE_STRATEGY = new PropertyDescriptor.Builder() + .name("Schema Write Strategy") + .description("Specifies how the schema for a Record should be added to the data.") + .allowableValues(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_CONTENT_ENCODED_SCHEMA) + .defaultValue(AVRO_SCHEMA_ATTRIBUTE.getValue()) + .required(true) + .build(); + + + private volatile ConfigurationContext configurationContext; + private volatile SchemaAccessWriter schemaAccessWriter; + + private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_CONTENT_ENCODED_SCHEMA)); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + + final AllowableValue[] strategies = getSchemaWriteStrategyValues().toArray(new AllowableValue[0]); + properties.add(new PropertyDescriptor.Builder() + .fromPropertyDescriptor(SCHEMA_WRITE_STRATEGY) + .defaultValue(getDefaultSchemaWriteStrategy().getValue()) + .allowableValues(strategies) + .build()); + + return properties; + } + + protected AllowableValue getDefaultSchemaWriteStrategy() { + return AVRO_SCHEMA_ATTRIBUTE; + } + + @OnEnabled + public void storeSchemaWriteStrategy(final ConfigurationContext context) { + this.configurationContext = context; + + final String writerValue = context.getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + this.schemaAccessWriter = getSchemaWriteStrategy(writerValue); + } + + @Override + protected ConfigurationContext getConfigurationContext() { + return configurationContext; + } + + protected SchemaAccessWriter getSchemaAccessWriter() { + return schemaAccessWriter; + } + + protected List getSchemaWriteStrategyValues() { + return strategyList; + } + + protected SchemaAccessWriter getSchemaWriteStrategy(final String allowableValue) { + if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_ATTRIBUTE.getValue())) { + return new SchemaNameAsAttribute(); + } else if (allowableValue.equalsIgnoreCase(AVRO_SCHEMA_ATTRIBUTE.getValue())) { + return new SchemaTextAsAttribute(); + } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) { + return new HortonworksEncodedSchemaReferenceWriter(); + } + + return null; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java new file mode 100644 index 000000000000..f4a6b7a74c12 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java @@ -0,0 +1,192 @@ +/* + * 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; + +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.avro.AvroSchemaValidator; +import org.apache.nifi.components.AllowableValue; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.schema.access.AvroSchemaTextStrategy; +import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceStrategy; +import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaNamePropertyStrategy; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; +import org.apache.nifi.serialization.record.RecordSchema; + +public abstract class SchemaRegistryService extends AbstractControllerService { + + static final AllowableValue SCHEMA_NAME_PROPERTY = new AllowableValue("schema-name", "Use 'Schema Name' Property", + "The name of the Schema to use is specified by the 'Schema Name' Property. The value of this property is used to lookup the Schema in the configured Schema Registry service."); + static final AllowableValue SCHEMA_TEXT_PROPERTY = new AllowableValue("schema-text-property", "Use 'Schema Text' Property", + "The text of the Schema itself is specified by the 'Schema Text' Property. The value of this property must be a valid Avro Schema. " + + "If Expression Language is used, the value of the 'Schema Text' property must be valid after substituting the expressions."); + static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference", + "The content of the FlowFile contains 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."); + + protected static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder() + .name("Schema Registry") + .description("Specifies the Controller Service to use for the Schema Registry") + .identifiesControllerService(SchemaRegistry.class) + .required(false) + .build(); + + protected static final PropertyDescriptor SCHEMA_ACCESS_STRATEGY = new PropertyDescriptor.Builder() + .name("Schema Access Strategy") + .description("Specifies how to obtain the schema that is to be used for interpreting the data.") + .allowableValues(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_CONTENT_ENCODED_SCHEMA) + .defaultValue(SCHEMA_TEXT_PROPERTY.getValue()) + .required(true) + .build(); + + static final PropertyDescriptor SCHEMA_NAME = new PropertyDescriptor.Builder() + .name("Schema Name") + .description("Specifies the name of the schema to lookup in the Schema Registry property") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(true) + .defaultValue("${schema.name}") + .required(true) + .build(); + + static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor.Builder() + .name("schema-text") + .displayName("Schema Text") + .description("The text of an Avro-formatted Schema") + .addValidator(new AvroSchemaValidator()) + .expressionLanguageSupported(true) + .defaultValue("${avro.schema}") + .required(true) + .build(); + + + private volatile ConfigurationContext configurationContext; + private volatile SchemaAccessStrategy schemaAccessStrategy; + + private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_CONTENT_ENCODED_SCHEMA)); + + + @Override + protected List getSupportedPropertyDescriptors() { + final List properties = new ArrayList<>(2); + + final AllowableValue[] strategies = getSchemaAccessStrategyValues().toArray(new AllowableValue[0]); + properties.add(new PropertyDescriptor.Builder() + .fromPropertyDescriptor(SCHEMA_ACCESS_STRATEGY) + .allowableValues(strategies) + .defaultValue(getDefaultSchemaAccessStrategy().getValue()) + .build()); + + properties.add(SCHEMA_REGISTRY); + properties.add(SCHEMA_NAME); + properties.add(SCHEMA_TEXT); + + return properties; + } + + protected AllowableValue getDefaultSchemaAccessStrategy() { + return SCHEMA_TEXT_PROPERTY; + } + + @OnEnabled + public void storeSchemaAccessStrategy(final ConfigurationContext context) { + this.configurationContext = context; + + final SchemaRegistry schemaRegistry = context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class); + + final PropertyDescriptor descriptor = getPropertyDescriptor(SCHEMA_ACCESS_STRATEGY.getName()); + final String schemaAccess = context.getProperty(descriptor).getValue(); + this.schemaAccessStrategy = getSchemaAccessStrategy(schemaAccess, schemaRegistry); + } + + protected ConfigurationContext getConfigurationContext() { + return configurationContext; + } + + protected SchemaAccessStrategy getSchemaAccessStrategy() { + return schemaAccessStrategy; + } + + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream) throws SchemaNotFoundException, IOException { + return getSchemaAccessStrategy().getSchema(flowFile, contentStream, configurationContext); + } + + private String getSchemaAccessStrategyName(final String schemaAccessValue) { + for (final AllowableValue allowableValue : SCHEMA_ACCESS_STRATEGY.getAllowableValues()) { + if (allowableValue.getValue().equalsIgnoreCase(schemaAccessValue)) { + return allowableValue.getDisplayName(); + } + } + + return null; + } + + private boolean isSchemaRegistryRequired(final String schemaAccessValue) { + return HWX_CONTENT_ENCODED_SCHEMA.getValue().equalsIgnoreCase(schemaAccessValue) || SCHEMA_NAME_PROPERTY.getValue().equalsIgnoreCase(schemaAccessValue); + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final String schemaAccessStrategy = validationContext.getProperty(SCHEMA_ACCESS_STRATEGY).getValue(); + if (isSchemaRegistryRequired(schemaAccessStrategy)) { + final boolean registrySet = validationContext.getProperty(SCHEMA_REGISTRY).isSet(); + if (!registrySet) { + final String schemaAccessStrategyName = getSchemaAccessStrategyName(schemaAccessStrategy); + + return Collections.singleton(new ValidationResult.Builder() + .subject("Schema Registry") + .explanation("The '" + schemaAccessStrategyName + "' Schema Access Strategy requires that the Schema Registry property be set.") + .valid(false) + .build()); + } + } + + return Collections.emptyList(); + } + + protected List getSchemaAccessStrategyValues() { + return strategyList; + } + + protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) { + if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue())) { + return new SchemaNamePropertyStrategy(schemaRegistry, getConfigurationContext().getProperty(SCHEMA_NAME)); + } else if (allowableValue.equalsIgnoreCase(SCHEMA_TEXT_PROPERTY.getValue())) { + return new AvroSchemaTextStrategy(getConfigurationContext().getProperty(SCHEMA_TEXT)); + } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) { + return new HortonworksEncodedSchemaReferenceStrategy(schemaRegistry); + } + + return null; + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java index 598a8c49d05b..cb6944411619 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextRecordSetWriter.java @@ -17,6 +17,7 @@ package org.apache.nifi.text; +import java.io.InputStream; import java.nio.charset.Charset; import java.util.ArrayList; import java.util.List; @@ -28,6 +29,7 @@ import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.serialization.RecordSetWriter; @@ -35,13 +37,12 @@ @Tags({"text", "freeform", "expression", "language", "el", "record", "recordset", "resultset", "writer", "serialize"}) @CapabilityDescription("Writes the contents of a RecordSet as free-form text. The configured " - + "text is able to make use of the Expression Language to reference each of the columns that are available " + + "text is able to make use of the Expression Language to reference each of the fields that are available " + "in a Record. Each record in the RecordSet will be separated by a single newline character.") public class FreeFormTextRecordSetWriter extends AbstractControllerService implements RecordSetWriterFactory { static final PropertyDescriptor TEXT = new PropertyDescriptor.Builder() .name("Text") - .description("The text to use when writing the results. This property will evaluate the Expression Language using any of the columns available to the Result Set. For example, if the " - + "following SQL Query is used: \"SELECT Name, COUNT(*) AS Count\" then the Expression can reference \"Name\" and \"Count\", such as \"${Name:toUpper()} ${Count:minus(1)}\"") + .description("The text to use when writing the results. This property will evaluate the Expression Language using any of the fields available in a Record.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(true) .required(true) @@ -73,7 +74,7 @@ public void onEnabled(final ConfigurationContext context) { } @Override - public RecordSetWriter createWriter(final ComponentLog logger) { + public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) { return new FreeFormTextWriter(textValue, characterSet); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java similarity index 94% rename from nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java rename to nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java index 56e2e3d44858..da9f70be5d5f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestAvroReaderWithEmbeddedSchema.java @@ -46,6 +46,7 @@ import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.MapRecord; @@ -55,11 +56,11 @@ import org.apache.nifi.serialization.record.RecordSchema; import org.junit.Test; -public class TestAvroRecordReader { +public class TestAvroReaderWithEmbeddedSchema { @Test - public void testLogicalTypes() throws IOException, ParseException, MalformedRecordException { + public void testLogicalTypes() throws IOException, ParseException, MalformedRecordException, SchemaNotFoundException { final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc")); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); @@ -92,7 +93,7 @@ public void testLogicalTypes() throws IOException, ParseException, MalformedReco } try (final InputStream in = new ByteArrayInputStream(serialized)) { - final AvroRecordReader reader = new AvroRecordReader(in); + final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in); final RecordSchema recordSchema = reader.getSchema(); assertEquals(RecordFieldType.TIME, recordSchema.getDataType("timeMillis").get().getFieldType()); @@ -113,8 +114,7 @@ public void testLogicalTypes() throws IOException, ParseException, MalformedReco } @Test - @SuppressWarnings({"unchecked", "rawtypes"}) - public void testDataTypes() throws IOException, MalformedRecordException { + public void testDataTypes() throws IOException, MalformedRecordException, SchemaNotFoundException { final List accountFields = new ArrayList<>(); accountFields.add(new Field("accountId", Schema.create(Type.LONG), null, (Object) null)); accountFields.add(new Field("accountName", Schema.create(Type.STRING), null, (Object) null)); @@ -164,12 +164,6 @@ public void testDataTypes() throws IOException, MalformedRecordException { map.put("greeting", "hello"); map.put("salutation", "good-bye"); - final List mapFields = new ArrayList<>(); - mapFields.add(new RecordField("greeting", RecordFieldType.STRING.getDataType())); - mapFields.add(new RecordField("salutation", RecordFieldType.STRING.getDataType())); - final RecordSchema mapSchema = new SimpleRecordSchema(mapFields); - final Record expectedRecord = new MapRecord(mapSchema, (Map) map); - final DatumWriter datumWriter = new GenericDatumWriter<>(schema); try (final DataFileWriter dataFileWriter = new DataFileWriter<>(datumWriter); final DataFileWriter writer = dataFileWriter.create(schema, baos)) { @@ -207,7 +201,7 @@ public void testDataTypes() throws IOException, MalformedRecordException { source = baos.toByteArray(); try (final InputStream in = new ByteArrayInputStream(source)) { - final AvroRecordReader reader = new AvroRecordReader(in); + final AvroRecordReader reader = new AvroReaderWithEmbeddedSchema(in); final RecordSchema recordSchema = reader.getSchema(); assertEquals(15, recordSchema.getFieldCount()); @@ -216,10 +210,10 @@ public void testDataTypes() throws IOException, MalformedRecordException { assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("balance").get().getFieldType()); assertEquals(RecordFieldType.FLOAT, recordSchema.getDataType("rate").get().getFieldType()); assertEquals(RecordFieldType.BOOLEAN, recordSchema.getDataType("debt").get().getFieldType()); - assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("nickname").get().getFieldType()); + assertEquals(RecordFieldType.STRING, recordSchema.getDataType("nickname").get().getFieldType()); assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("binary").get().getFieldType()); assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("fixed").get().getFieldType()); - assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("map").get().getFieldType()); + assertEquals(RecordFieldType.MAP, recordSchema.getDataType("map").get().getFieldType()); assertEquals(RecordFieldType.ARRAY, recordSchema.getDataType("array").get().getFieldType()); assertEquals(RecordFieldType.RECORD, recordSchema.getDataType("account").get().getFieldType()); assertEquals(RecordFieldType.DOUBLE, recordSchema.getDataType("desiredbalance").get().getFieldType()); @@ -237,7 +231,7 @@ public void testDataTypes() throws IOException, MalformedRecordException { assertEquals(null, values[5]); assertArrayEquals(toObjectArray("binary".getBytes(StandardCharsets.UTF_8)), (Object[]) values[6]); assertArrayEquals(toObjectArray("fixed".getBytes(StandardCharsets.UTF_8)), (Object[]) values[7]); - assertEquals(expectedRecord, values[8]); + assertEquals(map, values[8]); assertArrayEquals(new Object[] {1L, 2L}, (Object[]) values[9]); final Map accountValues = new HashMap<>(); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java index 21028139aa9a..f771a590a249 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java @@ -41,13 +41,10 @@ import java.util.TimeZone; import org.apache.avro.Schema; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.GenericData.Array; -import org.apache.avro.generic.GenericData.StringType; +import org.apache.avro.generic.GenericRecord; import org.apache.nifi.serialization.SimpleRecordSchema; +import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordField; @@ -56,12 +53,19 @@ import org.apache.nifi.serialization.record.RecordSet; import org.junit.Test; -public class TestWriteAvroResult { +public abstract class TestWriteAvroResult { + + protected abstract WriteAvroResult createWriter(Schema schema); + + protected abstract GenericRecord readRecord(InputStream in, Schema schema) throws IOException; + + protected void verify(final WriteResult writeResult) { + } @Test public void testLogicalTypes() throws IOException, ParseException { final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/logical-types.avsc")); - final WriteAvroResult writer = new WriteAvroResult(schema); + final WriteAvroResult writer = createWriter(schema); final List fields = new ArrayList<>(); fields.add(new RecordField("timeMillis", RecordFieldType.TIME.getDataType())); @@ -91,11 +95,7 @@ public void testLogicalTypes() throws IOException, ParseException { } try (final InputStream in = new ByteArrayInputStream(data)) { - final DataFileStream dataFileStream = new DataFileStream<>(in, new GenericDatumReader()); - final Schema avroSchema = dataFileStream.getSchema(); - GenericData.setStringType(avroSchema, StringType.String); - - final GenericRecord avroRecord = dataFileStream.next(); + final GenericRecord avroRecord = readRecord(in, schema); final long secondsSinceMidnight = 33 + (20 * 60) + (14 * 60 * 60); final long millisSinceMidnight = secondsSinceMidnight * 1000L; @@ -112,7 +112,7 @@ public void testLogicalTypes() throws IOException, ParseException { public void testDataTypes() throws IOException { // TODO: Test Enums final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/datatypes.avsc")); - final WriteAvroResult writer = new WriteAvroResult(schema); + final WriteAvroResult writer = createWriter(schema); final List subRecordFields = Collections.singletonList(new RecordField("field1", RecordFieldType.STRING.getDataType())); final RecordSchema subRecordSchema = new SimpleRecordSchema(subRecordFields); @@ -148,21 +148,18 @@ public void testDataTypes() throws IOException { final byte[] data; try (final ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - writer.write(RecordSet.of(record.getSchema(), record), baos); + final WriteResult writeResult = writer.write(RecordSet.of(record.getSchema(), record), baos); + verify(writeResult); data = baos.toByteArray(); } try (final InputStream in = new ByteArrayInputStream(data)) { - final DataFileStream dataFileStream = new DataFileStream<>(in, new GenericDatumReader()); - final Schema avroSchema = dataFileStream.getSchema(); - GenericData.setStringType(avroSchema, StringType.String); - - final GenericRecord avroRecord = dataFileStream.next(); + final GenericRecord avroRecord = readRecord(in, schema); assertMatch(record, avroRecord); } } - private void assertMatch(final Record record, final GenericRecord avroRecord) { + protected void assertMatch(final Record record, final GenericRecord avroRecord) { for (final String fieldName : record.getSchema().getFieldNames()) { Object avroValue = avroRecord.get(fieldName); final Object recordValue = record.getValue(fieldName); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java new file mode 100644 index 000000000000..6ace01224aea --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithSchema.java @@ -0,0 +1,46 @@ +/* + * 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.avro; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericData.StringType; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; + +public class TestWriteAvroResultWithSchema extends TestWriteAvroResult { + + @Override + protected WriteAvroResult createWriter(final Schema schema) { + return new WriteAvroResultWithSchema(schema); + } + + @Override + protected GenericRecord readRecord(final InputStream in, final Schema schema) throws IOException { + final DataFileStream dataFileStream = new DataFileStream<>(in, new GenericDatumReader()); + final Schema avroSchema = dataFileStream.getSchema(); + GenericData.setStringType(avroSchema, StringType.String); + final GenericRecord avroRecord = dataFileStream.next(); + + return avroRecord; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java new file mode 100644 index 000000000000..d40bb552be01 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResultWithoutSchema.java @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.nifi.avro; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Map; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; +import org.apache.nifi.schema.access.SchemaTextAsAttribute; +import org.apache.nifi.serialization.WriteResult; +import org.junit.Assert; + +public class TestWriteAvroResultWithoutSchema extends TestWriteAvroResult { + + @Override + protected WriteAvroResult createWriter(final Schema schema) { + return new WriteAvroResultWithExternalSchema(schema, AvroTypeUtil.createSchema(schema), new SchemaTextAsAttribute()); + } + + @Override + protected GenericRecord readRecord(final InputStream in, final Schema schema) throws IOException { + final BinaryDecoder decoder = DecoderFactory.get().binaryDecoder(in, null); + final GenericDatumReader reader = new GenericDatumReader(schema); + return reader.read(null, decoder); + } + + @Override + protected void verify(final WriteResult writeResult) { + final Map attributes = writeResult.getAttributes(); + + final String schemaText = attributes.get("avro.schema"); + Assert.assertNotNull(schemaText); + new Schema.Parser().parse(schemaText); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java new file mode 100644 index 000000000000..3eed784904ac --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestCSVHeaderSchemaStrategy.java @@ -0,0 +1,69 @@ +/* + * 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.csv; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.schema.access.SchemaNotFoundException; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.util.MockConfigurationContext; +import org.junit.Test; + +public class TestCSVHeaderSchemaStrategy { + + @Test + public void testSimple() throws SchemaNotFoundException, IOException { + final CSVHeaderSchemaStrategy strategy = new CSVHeaderSchemaStrategy(); + final String headerLine = "a, b, c, d, e\\,z, f"; + final byte[] headerBytes = headerLine.getBytes(); + + final Map properties = new HashMap<>(); + properties.put(CSVUtils.CSV_FORMAT, CSVUtils.CUSTOM.getValue()); + properties.put(CSVUtils.COMMENT_MARKER, "#"); + properties.put(CSVUtils.VALUE_SEPARATOR, ","); + properties.put(CSVUtils.TRIM_FIELDS, "true"); + properties.put(CSVUtils.QUOTE_CHAR, "\""); + properties.put(CSVUtils.ESCAPE_CHAR, "\\"); + + final ConfigurationContext context = new MockConfigurationContext(properties, null); + + final RecordSchema schema; + try (final InputStream bais = new ByteArrayInputStream(headerBytes)) { + schema = strategy.getSchema(null, bais, context); + } + + final List expectedFieldNames = Arrays.asList("a", "b", "c", "d", "e,z", "f"); + assertEquals(expectedFieldNames, schema.getFieldNames()); + + assertTrue(schema.getFields().stream() + .allMatch(field -> field.getDataType().equals(RecordFieldType.STRING.getDataType()))); + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java index 1e8997bc529d..2a7f617b5025 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java @@ -36,6 +36,7 @@ import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.QuoteMode; +import org.apache.nifi.schema.access.SchemaNameAsAttribute; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; @@ -52,7 +53,6 @@ public class TestWriteCSVResult { @Test public void testDataTypes() throws IOException { final CSVFormat csvFormat = CSVFormat.DEFAULT.withQuoteMode(QuoteMode.ALL).withRecordSeparator("\n"); - final WriteCSVResult result = new WriteCSVResult(csvFormat, RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); final StringBuilder headerBuilder = new StringBuilder(); final List fields = new ArrayList<>(); @@ -71,6 +71,9 @@ public void testDataTypes() throws IOException { } final RecordSchema schema = new SimpleRecordSchema(fields); + final WriteCSVResult result = new WriteCSVResult(csvFormat, schema, new SchemaNameAsAttribute(), + RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); + final long now = System.currentTimeMillis(); final Map valueMap = new HashMap<>(); valueMap.put("string", "string"); @@ -117,7 +120,7 @@ public void testDataTypes() throws IOException { expectedBuilder.append('"').append(dateValue).append('"').append(','); expectedBuilder.append('"').append(timeValue).append('"').append(','); expectedBuilder.append('"').append(timestampValue).append('"').append(','); - expectedBuilder.append(",\"48\","); + expectedBuilder.append(",\"48\",,"); final String expectedValues = expectedBuilder.toString(); assertEquals(expectedValues, values); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java index a741ad12055f..ae5d433effa3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/grok/TestGrokRecordReader.java @@ -45,7 +45,7 @@ public void testParseSingleLineLogMessages() throws GrokException, IOException, grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt"); grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}"); - final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null); + final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true); final String[] logLevels = new String[] {"INFO", "WARN", "ERROR", "FATAL", "FINE"}; final String[] messages = new String[] {"Test Message 1", "Red", "Green", "Blue", "Yellow"}; @@ -75,7 +75,7 @@ public void testParseEmptyMessageWithStackTrace() throws GrokException, IOExcept final String msg = "2016-08-04 13:26:32,473 INFO [Leader Election Notification Thread-1] o.a.n.LoggerClass \n" + "org.apache.nifi.exception.UnitTestException: Testing to ensure we are able to capture stack traces"; final InputStream bais = new ByteArrayInputStream(msg.getBytes(StandardCharsets.UTF_8)); - final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, null); + final GrokRecordReader deserializer = new GrokRecordReader(bais, grok, GrokReader.createRecordSchema(grok), true); final Object[] values = deserializer.nextRecord().getValues(); @@ -98,7 +98,7 @@ public void testParseNiFiSampleLog() throws IOException, GrokException, Malforme grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt"); grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}"); - final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null); + final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true); final String[] logLevels = new String[] {"INFO", "INFO", "INFO", "WARN", "WARN"}; @@ -122,7 +122,7 @@ public void testParseNiFiSampleMultilineWithStackTrace() throws IOException, Gro grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt"); grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} \\[%{DATA:thread}\\] %{DATA:class} %{GREEDYDATA:message}?"); - final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null); + final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true); final String[] logLevels = new String[] {"INFO", "INFO", "ERROR", "WARN", "WARN"}; @@ -154,7 +154,7 @@ public void testParseStackTrace() throws GrokException, IOException, MalformedRe grok.addPatternFromFile("src/main/resources/default-grok-patterns.txt"); grok.compile("%{TIMESTAMP_ISO8601:timestamp} %{LOGLEVEL:level} %{GREEDYDATA:message}"); - final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, null); + final GrokRecordReader deserializer = new GrokRecordReader(fis, grok, GrokReader.createRecordSchema(grok), true); final String[] logLevels = new String[] {"INFO", "ERROR", "INFO"}; final String[] messages = new String[] {"message without stack trace", diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java index 2422206c6399..75e4d31990eb 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestJsonTreeRowRecordReader.java @@ -309,4 +309,27 @@ public void testReadUnicodeCharacters() throws IOException, MalformedRecordExcep } } + @Test + public void testIncorrectSchema() throws IOException, MalformedRecordException { + final DataType accountType = RecordFieldType.RECORD.getRecordDataType(getAccountSchema()); + final List fields = getDefaultFields(); + fields.add(new RecordField("account", accountType)); + fields.remove(new RecordField("balance", RecordFieldType.DOUBLE.getDataType())); + + final RecordSchema schema = new SimpleRecordSchema(fields); + + try (final InputStream in = new FileInputStream(new File("src/test/resources/json/single-bank-account-wrong-field-type.json")); + final JsonTreeRowRecordReader reader = new JsonTreeRowRecordReader(in, Mockito.mock(ComponentLog.class), schema, dateFormat, timeFormat, timestampFormat)) { + + reader.nextRecord().getValues(); + Assert.fail("Was able to read record with invalid schema."); + + } catch (final MalformedRecordException mre) { + final String msg = mre.getCause().getMessage(); + assertTrue(msg.contains("account.balance")); + assertTrue(msg.contains("true")); + assertTrue(msg.contains("Double")); + assertTrue(msg.contains("Boolean")); + } + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java index 6119d36bf3f5..5c8bc493e3dd 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/json/TestWriteJsonResult.java @@ -37,6 +37,7 @@ import java.util.TimeZone; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNameAsAttribute; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; @@ -52,9 +53,6 @@ public class TestWriteJsonResult { @Test public void testDataTypes() throws IOException, ParseException { - final WriteJsonResult writer = new WriteJsonResult(Mockito.mock(ComponentLog.class), true, RecordFieldType.DATE.getDefaultFormat(), - RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); - final List fields = new ArrayList<>(); for (final RecordFieldType fieldType : RecordFieldType.values()) { if (fieldType == RecordFieldType.CHOICE) { @@ -63,16 +61,25 @@ public void testDataTypes() throws IOException, ParseException { possibleTypes.add(RecordFieldType.LONG.getDataType()); fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getChoiceDataType(possibleTypes))); + } else if (fieldType == RecordFieldType.MAP) { + fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getMapDataType(RecordFieldType.INT.getDataType()))); } else { fields.add(new RecordField(fieldType.name().toLowerCase(), fieldType.getDataType())); } } final RecordSchema schema = new SimpleRecordSchema(fields); + final WriteJsonResult writer = new WriteJsonResult(Mockito.mock(ComponentLog.class), schema, new SchemaNameAsAttribute(), true, RecordFieldType.DATE.getDefaultFormat(), + RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); + final DateFormat df = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss.SSS"); df.setTimeZone(TimeZone.getTimeZone("gmt")); final long time = df.parse("2017/01/01 17:00:00.000").getTime(); + final Map map = new LinkedHashMap<>(); + map.put("height", 48); + map.put("width", 96); + final Map valueMap = new LinkedHashMap<>(); valueMap.put("string", "string"); valueMap.put("boolean", true); @@ -90,6 +97,7 @@ public void testDataTypes() throws IOException, ParseException { valueMap.put("record", null); valueMap.put("array", null); valueMap.put("choice", 48L); + valueMap.put("map", map); final Record record = new MapRecord(schema, valueMap); final RecordSet rs = RecordSet.of(schema, record); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json index 40c28dd383dd..881925c162e6 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/output/dataTypes.json @@ -14,5 +14,9 @@ "timestamp" : "2017-01-01 17:00:00", "record" : null, "choice" : 48, - "array" : null + "array" : null, + "map" : { + "height" : 48, + "width" : 96 + } } ] \ 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/test/resources/json/single-bank-account-wrong-field-type.json b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json new file mode 100644 index 000000000000..50d676c591f1 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/resources/json/single-bank-account-wrong-field-type.json @@ -0,0 +1,13 @@ +{ + "id": 1, + "name": "John Doe", + "address": "123 My Street", + "city": "My City", + "state": "MS", + "zipCode": "11111", + "country": "USA", + "account": { + "id": 42, + "balance": true + } +} \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java index 68c246182ce5..5c20f58450f7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java @@ -16,38 +16,62 @@ */ package org.apache.nifi.schemaregistry.services; -import java.util.Map; +import java.io.IOException; import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.record.RecordSchema; /** * Represents {@link ControllerService} strategy to expose internal and/or * integrate with external Schema Registry */ -public interface SchemaRegistry extends ControllerService, AutoCloseable { - - public static final String SCHEMA_NAME_ATTR = "schema.name"; +public interface SchemaRegistry extends ControllerService { /** * Retrieves and returns the textual representation of the schema based on - * the provided name of the schema available in Schema Registry. Will throw - * an runtime exception if schema can not be found. + * the provided name of the schema available in Schema Registry. + * + * @return the text that corresponds to the latest version of the schema with the given name + * + * @throws IOException if unable to communicate with the backing store + * @throws SchemaNotFoundException if unable to find the schema with the given name */ - String retrieveSchemaText(String schemaName); + String retrieveSchemaText(String schemaName) throws IOException, SchemaNotFoundException; /** - * Retrieves and returns the textual representation of the schema based on - * the provided name of the schema available in Schema Registry and optional - * additional attributes. Will throw an runtime exception if schema can not - * be found. + * Retrieves the textual representation of the schema with the given ID and version + * + * @param schemaId the unique identifier for the desired schema + * @param version the version of the desired schema + * @return the textual representation of the schema with the given ID and version + * + * @throws IOException if unable to communicate with the backing store + * @throws SchemaNotFoundException if unable to find the schema with the given id and version */ - String retrieveSchemaText(String schemaName, Map attributes); + String retrieveSchemaText(long schemaId, int version) throws IOException, SchemaNotFoundException; + /** + * Retrieves and returns the RecordSchema based on the provided name of the schema available in Schema Registry. + * + * @return the latest version of the schema with the given name, or null if no schema can be found with the given name. + * @throws SchemaNotFoundException if unable to find the schema with the given name + */ + RecordSchema retrieveSchema(String schemaName) throws IOException, SchemaNotFoundException; - RecordSchema retrieveSchema(String schemaName); + /** + * Retrieves the schema with the given ID and version + * + * @param schemaId the unique identifier for the desired schema + * @param version the version of the desired schema + * @return the schema with the given ID and version or null if no schema + * can be found with the given ID and version + * + * @throws IOException if unable to communicate with the backing store + * @throws SchemaNotFoundException if unable to find the schema with the given id and version + */ + RecordSchema retrieveSchema(long schemaId, int version) throws IOException, SchemaNotFoundException; - RecordSchema retrieveSchema(String schemaName, Map attributes); } From 5a7020dd207cf0cf6c0f7144567618f7645e63ea Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Wed, 19 Apr 2017 17:03:20 -0400 Subject: [PATCH 2/8] NIFI-3682: Rebased against master and addressed issues caused by changing API --- .../processors/standard/ConvertRecord.java | 20 ++++++++++++++----- 1 file changed, 15 insertions(+), 5 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java index 9a505a2284a2..2b2caa4e8e04 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ConvertRecord.java @@ -17,6 +17,7 @@ package org.apache.nifi.processors.standard; +import java.io.BufferedInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -46,11 +47,12 @@ import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.io.StreamCallback; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.RecordSetWriterFactory; -import org.apache.nifi.serialization.RowRecordReaderFactory; import org.apache.nifi.serialization.WriteResult; @EventDriven @@ -74,7 +76,7 @@ public class ConvertRecord extends AbstractProcessor { .name("record-reader") .displayName("Record Reader") .description("Specifies the Controller Service to use for reading incoming data") - .identifiesControllerService(RowRecordReaderFactory.class) + .identifiesControllerService(RecordReaderFactory.class) .required(true) .build(); static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() @@ -118,9 +120,17 @@ public void onTrigger(final ProcessContext context, final ProcessSession session return; } - final RowRecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RowRecordReaderFactory.class); + final RecordReaderFactory readerFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); final RecordSetWriterFactory writerFactory = context.getProperty(RECORD_WRITER).asControllerService(RecordSetWriterFactory.class); - final RecordSetWriter writer = writerFactory.createWriter(getLogger()); + final RecordSetWriter writer; + try (final InputStream rawIn = session.read(flowFile); + final InputStream in = new BufferedInputStream(rawIn)) { + writer = writerFactory.createWriter(getLogger(), flowFile, in); + } catch (final Exception e) { + getLogger().error("Failed to convert records for {}; will route to failure", new Object[] {flowFile, e}); + session.transfer(flowFile, REL_FAILURE); + return; + } final AtomicReference writeResultRef = new AtomicReference<>(); @@ -134,7 +144,7 @@ public void process(final InputStream in, final OutputStream out) throws IOExcep final WriteResult writeResult = writer.write(reader.createRecordSet(), out); writeResultRef.set(writeResult); - } catch (final MalformedRecordException e) { + } catch (final SchemaNotFoundException | MalformedRecordException e) { throw new ProcessException("Could not parse incoming data", e); } } From 84bcac6511d838b98e15519f559c20a029f26d28 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Thu, 20 Apr 2017 11:50:46 -0400 Subject: [PATCH 3/8] NIFI-3682: Addressed feedback from PR review; added config options for CSV reader/writer to optionally include first (header) line --- .../apache/nifi/util/MockProcessSession.java | 12 ++++++-- .../apache/nifi/util/MockSessionFactory.java | 6 ++-- .../util/StandardProcessorTestRunner.java | 13 ++++++-- .../java/org/apache/nifi/util/TestRunner.java | 16 ++++++++++ .../nifi/processors/standard/QueryRecord.java | 6 +++- .../processors/standard/TestQueryRecord.java | 30 +++++++++++++++++++ .../util/record/MockRecordParser.java | 3 +- .../util/record/MockRecordWriter.java | 6 ++-- .../java/org/apache/nifi/avro/AvroReader.java | 4 +-- .../java/org/apache/nifi/csv/CSVReader.java | 1 + .../org/apache/nifi/csv/CSVRecordReader.java | 8 ++--- .../apache/nifi/csv/CSVRecordSetWriter.java | 5 +++- .../java/org/apache/nifi/csv/CSVUtils.java | 28 ++++++++++++++--- .../org/apache/nifi/csv/WriteCSVResult.java | 6 ++-- .../schema/access/AvroSchemaTextStrategy.java | 2 +- .../apache/nifi/csv/TestWriteCSVResult.java | 2 +- 16 files changed, 122 insertions(+), 26 deletions(-) diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java index faf6e42988b4..f05b9b35ba9d 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockProcessSession.java @@ -72,6 +72,7 @@ public class MockProcessSession implements ProcessSession { private final Map counterMap = new HashMap<>(); private final Set recursionSet = new HashSet<>(); private final MockProvenanceReporter provenanceReporter; + private final boolean enforceReadStreamsClosed; // A List of InputStreams that have been created by calls to {@link #read(FlowFile)} and have not yet been closed. private final Map openInputStreams = new HashMap<>(); @@ -83,7 +84,12 @@ public class MockProcessSession implements ProcessSession { private static final AtomicLong enqueuedIndex = new AtomicLong(0L); public MockProcessSession(final SharedSessionState sharedState, final Processor processor) { + this(sharedState, processor, true); + } + + public MockProcessSession(final SharedSessionState sharedState, final Processor processor, final boolean enforceReadStreamsClosed) { this.processor = processor; + this.enforceReadStreamsClosed = enforceReadStreamsClosed; this.sharedState = sharedState; this.processorQueue = sharedState.getFlowFileQueue(); provenanceReporter = new MockProvenanceReporter(this, sharedState, processor.getIdentifier(), processor.getClass().getSimpleName()); @@ -218,8 +224,10 @@ public void commit() { } } - throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via " - + "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy); + if (enforceReadStreamsClosed) { + throw new FlowFileHandlingException("Cannot commit session because the following Input Streams were created via " + + "calls to ProcessSession.read(FlowFile) and never closed: " + openStreamCopy); + } } committed = true; diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java b/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java index 49b879641c77..010cc97646db 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/MockSessionFactory.java @@ -29,15 +29,17 @@ public class MockSessionFactory implements ProcessSessionFactory { private final Processor processor; private final SharedSessionState sharedState; private final Set createdSessions = new CopyOnWriteArraySet<>(); + private final boolean enforceReadStreamsClosed; - MockSessionFactory(final SharedSessionState sharedState, final Processor processor) { + MockSessionFactory(final SharedSessionState sharedState, final Processor processor, final boolean enforceReadStreamsClosed) { this.sharedState = sharedState; this.processor = processor; + this.enforceReadStreamsClosed = enforceReadStreamsClosed; } @Override public ProcessSession createSession() { - final MockProcessSession session = new MockProcessSession(sharedState, processor); + final MockProcessSession session = new MockProcessSession(sharedState, processor, enforceReadStreamsClosed); createdSessions.add(session); return session; } diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java index 451eea93fdaf..6c356430b22d 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/StandardProcessorTestRunner.java @@ -78,7 +78,6 @@ public class StandardProcessorTestRunner implements TestRunner { private final Processor processor; private final MockProcessContext context; private final MockFlowFileQueue flowFileQueue; - private final MockSessionFactory sessionFactory; private final SharedSessionState sharedState; private final AtomicLong idGenerator; private final boolean triggerSerially; @@ -87,17 +86,19 @@ public class StandardProcessorTestRunner implements TestRunner { private final MockVariableRegistry variableRegistry; private int numThreads = 1; + private MockSessionFactory sessionFactory; private final AtomicInteger invocations = new AtomicInteger(0); private final Map controllerServiceLoggers = new HashMap<>(); private final MockComponentLog logger; + private boolean enforceReadStreamsClosed = true; StandardProcessorTestRunner(final Processor processor) { this.processor = processor; this.idGenerator = new AtomicLong(0L); this.sharedState = new SharedSessionState(processor, idGenerator); this.flowFileQueue = sharedState.getFlowFileQueue(); - this.sessionFactory = new MockSessionFactory(sharedState, processor); + this.sessionFactory = new MockSessionFactory(sharedState, processor, enforceReadStreamsClosed); this.processorStateManager = new MockStateManager(processor); this.variableRegistry = new MockVariableRegistry(); this.context = new MockProcessContext(processor, processorStateManager, variableRegistry); @@ -117,6 +118,12 @@ public class StandardProcessorTestRunner implements TestRunner { ReflectionUtils.quietlyInvokeMethodsWithAnnotation(OnConfigurationRestored.class, processor); } + @Override + public void enforceReadStreamsClosed(final boolean enforce) { + enforceReadStreamsClosed = enforce; + this.sessionFactory = new MockSessionFactory(sharedState, processor, enforceReadStreamsClosed); + } + @Override public void setValidateExpressionUsage(final boolean validate) { context.setValidateExpressionUsage(validate); @@ -412,7 +419,7 @@ public MockFlowFile enqueue(final InputStream data) { @Override public MockFlowFile enqueue(final InputStream data, final Map attributes) { - final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor); + final MockProcessSession session = new MockProcessSession(new SharedSessionState(processor, idGenerator), processor, enforceReadStreamsClosed); MockFlowFile flowFile = session.create(); flowFile = session.importFrom(data, flowFile); flowFile = session.putAllAttributes(flowFile, attributes); diff --git a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java index 1c014c3f5973..9a1a10d24205 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java +++ b/nifi-mock/src/main/java/org/apache/nifi/util/TestRunner.java @@ -30,6 +30,7 @@ import org.apache.nifi.controller.queue.QueueSize; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.ProcessSessionFactory; import org.apache.nifi.processor.Processor; import org.apache.nifi.processor.Relationship; @@ -949,4 +950,19 @@ public interface TestRunner { * @param predicate conditions */ void assertAllConditionsMet(final Relationship relationship, Predicate predicate); + + /** + * By default, if {@link ProcessSession#read(FlowFile)} is called, the InputStream that is returned MUST be closed by + * the processor under test or calls to {@link ProcessSession#commit()} will throw an Exception. This method allows + * the developer to indicate explicitly that they do or do not want this functionality. The ProcessSession that is used + * in the framework when running NiFi does not enforce this, as the framework itself tracks the InputStreams that it returns + * and ensures that they are properly closed on session commit or rollback. However, it is considered a best practice for + * Processors to close the streams themselves whenever they are no longer needed. There may be cases, however, where this + * is not feasible or easy and this method provides developers the ability to indicate that by disabling enforcement so that + * the framework will handle this. + * + * @param enforce true if calls to session.commit() should fail if the read streams are not properly closed. + */ + void enforceReadStreamsClosed(boolean enforce); + } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java index 3de0a79307a8..59829087cc2a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryRecord.java @@ -268,6 +268,7 @@ public void onTrigger(final ProcessContext context, final ProcessSession session // and we cannot call session.read() on the original FlowFile while we are within a write // callback for the original FlowFile. FlowFile transformed = session.create(original); + boolean flowFileRemoved = false; try { final String sql = context.getProperty(descriptor).evaluateAttributeExpressions(original).getValue(); @@ -300,6 +301,7 @@ public void process(final OutputStream out) throws IOException { final WriteResult result = writeResultRef.get(); if (result.getRecordCount() == 0 && !context.getProperty(INCLUDE_ZERO_RECORD_FLOWFILES).asBoolean()) { session.remove(transformed); + flowFileRemoved = true; transformedFlowFiles.remove(transformed); getLogger().info("Transformed {} but the result contained no data so will not pass on a FlowFile", new Object[] {original}); } else { @@ -317,7 +319,9 @@ public void process(final OutputStream out) throws IOException { } } finally { // Ensure that we have the FlowFile in the set in case we throw any Exception - createdFlowFiles.add(transformed); + if (!flowFileRemoved) { + createdFlowFiles.add(transformed); + } } } diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java index 65002c68c468..32c3635384a6 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestQueryRecord.java @@ -154,6 +154,36 @@ public void testTransformCalc() throws InitializationException, IOException, SQL out.assertContentEquals("\"NAME\",\"POINTS\"\n\"100\",\"90.75\"\n"); } + @Test + public void testHandlingWithInvalidSchema() throws InitializationException { + final MockRecordParser parser = new MockRecordParser(); + parser.addSchemaField("name", RecordFieldType.STRING); + parser.addSchemaField("favorite_color", RecordFieldType.STRING); + parser.addSchemaField("address", RecordFieldType.STRING); + parser.addRecord("Tom", "blue", null); + parser.addRecord("Jerry", "red", null); + + final MockRecordWriter writer = new MockRecordWriter("\"name\",\"points\""); + + final TestRunner runner = TestRunners.newTestRunner(QueryRecord.class); + runner.enforceReadStreamsClosed(false); + runner.addControllerService("parser", parser); + runner.enableControllerService(parser); + runner.addControllerService("writer", writer); + runner.enableControllerService(writer); + + runner.setProperty(QueryRecord.INCLUDE_ZERO_RECORD_FLOWFILES, "false"); + runner.setProperty("rel1", "select * from FLOWFILE where address IS NOT NULL"); + runner.setProperty("rel2", "select name, CAST(favorite_color AS DOUBLE) AS num from FLOWFILE"); + runner.setProperty("rel3", "select * from FLOWFILE where address IS NOT NULL"); + runner.setProperty(QueryRecord.RECORD_READER_FACTORY, "parser"); + runner.setProperty(QueryRecord.RECORD_WRITER_FACTORY, "writer"); + + runner.enqueue(""); + runner.run(); + + runner.assertAllFlowFilesTransferred(QueryRecord.REL_FAILURE, 1); + } @Test public void testAggregateFunction() throws InitializationException, IOException { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java index e78fddd53667..fcf0d1044741 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordParser.java @@ -28,6 +28,7 @@ import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; import org.apache.nifi.serialization.RecordReaderFactory; @@ -61,7 +62,7 @@ public void addRecord(Object... values) { } @Override - public RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws IOException { + public RecordReader createRecordReader(FlowFile flowFile, InputStream in, ComponentLog logger) throws IOException, SchemaNotFoundException { final Iterator itr = records.iterator(); return new RecordReader() { diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java index df3e0f18c84b..1dbfd042248b 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/util/record/MockRecordWriter.java @@ -71,9 +71,11 @@ public WriteResult write(final RecordSet rs, final OutputStream out) throws IOEx final String val = record.getAsString(fieldName); if (quoteValues) { out.write("\"".getBytes()); - out.write(val.getBytes()); + if (val != null) { + out.write(val.getBytes()); + } out.write("\"".getBytes()); - } else { + } else if (val != null) { out.write(val.getBytes()); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java index 8b5944b3a7ce..f5b4373b1beb 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroReader.java @@ -34,8 +34,8 @@ import org.apache.nifi.serialization.SchemaRegistryService; @Tags({"avro", "parse", "record", "row", "reader", "delimited", "comma", "separated", "values"}) -@CapabilityDescription("Parses Avro data and returns each Avro record as an separate Record object. The Avro data must contain " - + "the schema itself.") +@CapabilityDescription("Parses Avro data and returns each Avro record as an separate Record object. The Avro data may contain the schema itself, " + + "or the schema can be externalized and accessed by one of the methods offered by the 'Schema Access Strategy' property.") public class AvroReader extends SchemaRegistryService implements RecordReaderFactory { private final AllowableValue EMBEDDED_AVRO_SCHEMA = new AllowableValue("embedded-avro-schema", "Use Embedded Avro Schema", "The FlowFile has the Avro Schema embedded within the content, and this schema will be used."); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java index 789b1eb1b62b..fed52f5a761a 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java @@ -67,6 +67,7 @@ protected List getSupportedPropertyDescriptors() { properties.add(DateTimeUtils.TIMESTAMP_FORMAT); properties.add(CSVUtils.CSV_FORMAT); properties.add(CSVUtils.VALUE_SEPARATOR); + properties.add(CSVUtils.SKIP_HEADER_LINE); properties.add(CSVUtils.QUOTE_CHAR); properties.add(CSVUtils.ESCAPE_CHAR); properties.add(CSVUtils.COMMENT_MARKER); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java index 161624551fd7..e80057f7418d 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java @@ -48,14 +48,14 @@ public class CSVRecordReader implements RecordReader { public CSVRecordReader(final InputStream in, final ComponentLog logger, final RecordSchema schema, final CSVFormat csvFormat, final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException { - // TODO: Need to make sure that we use correct logic for skipping header line. - final Reader reader = new InputStreamReader(new BOMInputStream(in)); - csvParser = new CSVParser(reader, csvFormat); - this.schema = schema; this.dateFormat = dateFormat; this.timeFormat = timeFormat; this.timestampFormat = timestampFormat; + + final Reader reader = new InputStreamReader(new BOMInputStream(in)); + final CSVFormat withHeader = csvFormat.withHeader(schema.getFieldNames().toArray(new String[0])); + csvParser = new CSVParser(reader, withHeader); } @Override diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java index 5d22afb77a94..f1d7fecde037 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java @@ -42,12 +42,14 @@ public class CSVRecordSetWriter extends DateTimeTextRecordSetWriter implements RecordSetWriterFactory { private volatile CSVFormat csvFormat; + private volatile boolean includeHeader; @Override protected List getSupportedPropertyDescriptors() { final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); properties.add(CSVUtils.CSV_FORMAT); properties.add(CSVUtils.VALUE_SEPARATOR); + properties.add(CSVUtils.INCLUDE_HEADER_LINE); properties.add(CSVUtils.QUOTE_CHAR); properties.add(CSVUtils.ESCAPE_CHAR); properties.add(CSVUtils.COMMENT_MARKER); @@ -62,11 +64,12 @@ protected List getSupportedPropertyDescriptors() { @OnEnabled public void storeCsvFormat(final ConfigurationContext context) { this.csvFormat = CSVUtils.createCSVFormat(context); + this.includeHeader = context.getProperty(CSVUtils.INCLUDE_HEADER_LINE).asBoolean(); } @Override public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws SchemaNotFoundException, IOException { final RecordSchema schema = getSchema(flowFile, in); - return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(), getDateFormat(), getTimeFormat(), getTimestampFormat()); + return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(), getDateFormat(), getTimeFormat(), getTimestampFormat(), includeHeader); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java index c63a765d5266..1048d219a971 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVUtils.java @@ -61,6 +61,18 @@ public class CSVUtils { .defaultValue("\"") .required(true) .build(); + static final PropertyDescriptor SKIP_HEADER_LINE = new PropertyDescriptor.Builder() + .name("Skip Header Line") + .description("Specifies whether or not the first line of CSV should be considered a Header and skipped. If the Schema Access Strategy " + + "indicates that the columns must be defined in the header, then this property will be ignored, since the header must always be " + + "present and won't be processed as a Record. Otherwise, this property should be 'true' if the first non-comment line of CSV " + + "contains header information that needs to be ignored.") + .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(false) + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); static final PropertyDescriptor COMMENT_MARKER = new PropertyDescriptor.Builder() .name("Comment Marker") .description("The character that is used to denote the start of a comment. Any line that begins with this comment will be ignored.") @@ -124,7 +136,13 @@ public class CSVUtils { .defaultValue("\\n") .required(true) .build(); - + static final PropertyDescriptor INCLUDE_HEADER_LINE = new PropertyDescriptor.Builder() + .name("Include Header Line") + .description("Specifies whether or not the CSV column names should be written out as the first line.") + .allowableValues("true", "false") + .defaultValue("true") + .required(true) + .build(); static CSVFormat createCSVFormat(final ConfigurationContext context) { final String formatName = context.getProperty(CSV_FORMAT).getValue(); @@ -156,9 +174,11 @@ private static CSVFormat buildCustomFormat(final ConfigurationContext context) { final char valueSeparator = getChar(context, VALUE_SEPARATOR); CSVFormat format = CSVFormat.newFormat(valueSeparator) .withAllowMissingColumnNames() - .withIgnoreEmptyLines() - // TODO: Need property to specify whether or not to skip first line as header! - .withFirstRecordAsHeader(); + .withIgnoreEmptyLines(); + + if (context.getProperty(SKIP_HEADER_LINE).asBoolean()) { + format = format.withFirstRecordAsHeader(); + } format = format.withQuote(getChar(context, QUOTE_CHAR)); format = format.withEscape(getChar(context, ESCAPE_CHAR)); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java index a3f15bd8212e..ec43dce40cde 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java @@ -41,15 +41,17 @@ public class WriteCSVResult implements RecordSetWriter { private final String dateFormat; private final String timeFormat; private final String timestampFormat; + private final boolean includeHeaderLine; public WriteCSVResult(final CSVFormat csvFormat, final RecordSchema recordSchema, final SchemaAccessWriter schemaWriter, - final String dateFormat, final String timeFormat, final String timestampFormat) { + final String dateFormat, final String timeFormat, final String timestampFormat, final boolean includeHeaderLine) { this.csvFormat = csvFormat; this.recordSchema = recordSchema; this.schemaWriter = schemaWriter; this.dateFormat = dateFormat; this.timeFormat = timeFormat; this.timestampFormat = timestampFormat; + this.includeHeaderLine = includeHeaderLine; } private String getFormat(final Record record, final String fieldName) { @@ -76,7 +78,7 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I int count = 0; final String[] columnNames = recordSchema.getFieldNames().toArray(new String[0]); - final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames).withSkipHeaderRecord(false); + final CSVFormat formatWithHeader = csvFormat.withHeader(columnNames).withSkipHeaderRecord(!includeHeaderLine); schemaWriter.writeHeader(recordSchema, rawOut); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java index 2b8d9ba74a67..c1676680f81d 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java @@ -39,7 +39,7 @@ public AvroSchemaTextStrategy(final PropertyValue schemaTextPropertyValue) { @Override public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { final String schemaText = schemaTextPropertyValue.evaluateAttributeExpressions(flowFile).getValue(); - if (schemaText == null) { + if (schemaText == null || schemaText.trim().isEmpty()) { throw new SchemaNotFoundException("FlowFile did not contain appropriate attributes to determine Schema Text"); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java index 2a7f617b5025..9424e79ef24f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/csv/TestWriteCSVResult.java @@ -72,7 +72,7 @@ public void testDataTypes() throws IOException { final RecordSchema schema = new SimpleRecordSchema(fields); final WriteCSVResult result = new WriteCSVResult(csvFormat, schema, new SchemaNameAsAttribute(), - RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat()); + RecordFieldType.DATE.getDefaultFormat(), RecordFieldType.TIME.getDefaultFormat(), RecordFieldType.TIMESTAMP.getDefaultFormat(), true); final long now = System.currentTimeMillis(); final Map valueMap = new HashMap<>(); From 7c6614f367b4bdb0b99dc8330af10272f1b15bec Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 21 Apr 2017 12:09:00 -0400 Subject: [PATCH 4/8] NIFI-3682: Implemented 'aliases' and 'default values' for schemas --- .../services/AvroSchemaRegistry.java | 6 +- .../serialization/SimpleRecordSchema.java | 24 ++- .../nifi/serialization/record/MapRecord.java | 82 +++++++- .../nifi/serialization/record/Record.java | 4 + .../serialization/record/RecordField.java | 47 ++++- .../serialization/record/RecordSchema.java | 6 + .../record/util/DataTypeUtils.java | 2 +- .../serialization/TestSimpleRecordSchema.java | 79 ++++++++ .../serialization/record/TestMapRecord.java | 188 ++++++++++++++++++ .../apache/nifi/avro/AvroRecordReader.java | 15 +- .../org/apache/nifi/avro/AvroTypeUtil.java | 5 +- .../org/apache/nifi/avro/WriteAvroResult.java | 17 +- .../java/org/apache/nifi/csv/CSVReader.java | 5 + .../org/apache/nifi/csv/CSVRecordReader.java | 21 +- .../org/apache/nifi/csv/WriteCSVResult.java | 19 +- .../apache/nifi/grok/GrokRecordReader.java | 17 +- .../nifi/json/JsonPathRowRecordReader.java | 17 +- .../nifi/json/JsonTreeRowRecordReader.java | 11 +- .../org/apache/nifi/json/WriteJsonResult.java | 6 +- ...onworksEncodedSchemaReferenceStrategy.java | 3 + ...rtonworksEncodedSchemaReferenceWriter.java | 3 + .../apache/nifi/text/FreeFormTextWriter.java | 23 ++- .../apache/nifi/avro/TestWriteAvroResult.java | 3 +- 23 files changed, 541 insertions(+), 62 deletions(-) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java index a5964c4bff59..0eca4ea472a0 100644 --- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java +++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java @@ -127,7 +127,8 @@ private RecordSchema createRecordSchema(final Schema avroSchema, final String te for (final Field field : avroSchema.getFields()) { final String fieldName = field.name(); final DataType dataType = determineDataType(field.schema()); - recordFields.add(new RecordField(fieldName, dataType)); + + recordFields.add(new RecordField(fieldName, dataType, field.defaultVal(), field.aliases())); } final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, text, "avro", SchemaIdentifier.ofName(schemaName)); @@ -185,7 +186,8 @@ private DataType determineDataType(final Schema avroSchema) { final String fieldName = field.name(); final Schema fieldSchema = field.schema(); final DataType fieldType = determineDataType(fieldSchema); - recordFields.add(new RecordField(fieldName, fieldType)); + + recordFields.add(new RecordField(fieldName, fieldType, field.defaultVal(), field.aliases())); } final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), "avro", SchemaIdentifier.EMPTY); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java index 576fda92b287..f367e1b72223 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java @@ -61,7 +61,19 @@ public SimpleRecordSchema(final List fields, final String text, fin int index = 0; for (final RecordField field : fields) { - fieldIndices.put(field.getFieldName(), index++); + Integer previousValue = fieldIndices.put(field.getFieldName(), index); + if (previousValue != null) { + throw new IllegalArgumentException("Two fields are given with the same name (or alias) of '" + field.getFieldName() + "'"); + } + + for (final String alias : field.getAliases()) { + previousValue = fieldIndices.put(alias, index); + if (previousValue != null) { + throw new IllegalArgumentException("Two fields are given with the same name (or alias) of '" + field.getFieldName() + "'"); + } + } + + index++; } } @@ -113,6 +125,16 @@ public Optional getDataType(final String fieldName) { return idx.isPresent() ? Optional.of(fields.get(idx.getAsInt()).getDataType()) : Optional.empty(); } + @Override + public Optional getField(final String fieldName) { + final OptionalInt indexOption = getFieldIndex(fieldName); + if (indexOption.isPresent()) { + return Optional.of(fields.get(indexOption.getAsInt())); + } + + return Optional.empty(); + } + private OptionalInt getFieldIndex(final String fieldName) { final Integer index = fieldIndices.get(fieldName); return index == null ? OptionalInt.empty() : OptionalInt.of(index); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java index 750bd98b29d3..56cf9099a818 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/MapRecord.java @@ -42,15 +42,86 @@ public RecordSchema getSchema() { public Object[] getValues() { final Object[] values = new Object[schema.getFieldCount()]; int i = 0; - for (final String fieldName : schema.getFieldNames()) { - values[i++] = getValue(fieldName); + for (final RecordField recordField : schema.getFields()) { + values[i++] = getValue(recordField); } return values; } @Override public Object getValue(final String fieldName) { - return values.get(fieldName); + final Optional fieldOption = schema.getField(fieldName); + if (fieldOption.isPresent()) { + return getValue(fieldOption.get()); + } + + return null; + } + + @Override + public Object getValue(final RecordField field) { + Object explicitValue = getExplicitValue(field); + if (explicitValue != null) { + return explicitValue; + } + + final Optional resolvedField = resolveField(field); + final boolean resolvedFieldDifferent = resolvedField.isPresent() && !resolvedField.get().equals(field); + if (resolvedFieldDifferent) { + explicitValue = getExplicitValue(resolvedField.get()); + if (explicitValue != null) { + return explicitValue; + } + } + + Object defaultValue = field.getDefaultValue(); + if (defaultValue != null) { + return defaultValue; + } + + if (resolvedFieldDifferent) { + return resolvedField.get().getDefaultValue(); + } + + return null; + } + + private Optional resolveField(final RecordField field) { + Optional resolved = schema.getField(field.getFieldName()); + if (resolved.isPresent()) { + return resolved; + } + + for (final String alias : field.getAliases()) { + resolved = schema.getField(alias); + if (resolved.isPresent()) { + return resolved; + } + } + + return Optional.empty(); + } + + private Object getExplicitValue(final RecordField field) { + final String canonicalFieldName = field.getFieldName(); + + // We use containsKey here instead of just calling get() and checking for a null value + // because if the true field name is set to null, we want to return null, rather than + // what the alias points to. Likewise for a specific alias, since aliases are defined + // in a List with a specific ordering. + Object value = values.get(canonicalFieldName); + if (value != null) { + return value; + } + + for (final String alias : field.getAliases()) { + value = values.get(alias); + if (value != null) { + return value; + } + } + + return null; } @Override @@ -68,6 +139,11 @@ public String getAsString(final String fieldName, final String format) { return convertToString(getValue(fieldName), format); } + @Override + public String getAsString(final RecordField field, final String format) { + return convertToString(getValue(field), format); + } + private String getFormat(final String optionalFormat, final RecordFieldType fieldType) { return (optionalFormat == null) ? fieldType.getDefaultFormat() : optionalFormat; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java index e1d52e90ccad..5e5e7badb8e1 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/Record.java @@ -38,10 +38,14 @@ public interface Record { Object getValue(String fieldName); + Object getValue(RecordField field); + String getAsString(String fieldName); String getAsString(String fieldName, String format); + String getAsString(RecordField field, String format); + Long getAsLong(String fieldName); Integer getAsInt(String fieldName); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java index 135ae66f0421..fe3d8e56438a 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordField.java @@ -17,29 +17,66 @@ package org.apache.nifi.serialization.record; +import java.util.Collections; +import java.util.Objects; +import java.util.Set; + +import org.apache.nifi.serialization.record.util.DataTypeUtils; + public class RecordField { private final String fieldName; private final DataType dataType; + private final Set aliases; + private final Object defaultValue; public RecordField(final String fieldName, final DataType dataType) { - this.fieldName = fieldName; - this.dataType = dataType; + this(fieldName, dataType, null, Collections.emptySet()); + } + + public RecordField(final String fieldName, final DataType dataType, final Object defaultValue) { + this(fieldName, dataType, defaultValue, Collections.emptySet()); + } + + public RecordField(final String fieldName, final DataType dataType, final Set aliases) { + this(fieldName, dataType, null, aliases); + } + + public RecordField(final String fieldName, final DataType dataType, final Object defaultValue, final Set aliases) { + if (defaultValue != null && !DataTypeUtils.isCompatibleDataType(defaultValue, dataType)) { + throw new IllegalArgumentException("Cannot set the default value for field [" + fieldName + "] to [" + defaultValue + + "] because that is not a valid value for Data Type [" + dataType + "]"); + } + + this.fieldName = Objects.requireNonNull(fieldName); + this.dataType = Objects.requireNonNull(dataType); + this.aliases = Collections.unmodifiableSet(Objects.requireNonNull(aliases)); + this.defaultValue = defaultValue; } public String getFieldName() { return fieldName; } + public Set getAliases() { + return aliases; + } + public DataType getDataType() { return dataType; } + public Object getDefaultValue() { + return defaultValue; + } + @Override public int hashCode() { final int prime = 31; int result = 1; - result = prime * result + ((dataType == null) ? 0 : dataType.hashCode()); - result = prime * result + ((fieldName == null) ? 0 : fieldName.hashCode()); + result = prime * result + dataType.hashCode(); + result = prime * result + fieldName.hashCode(); + result = prime * result + aliases.hashCode(); + result = prime * result + ((defaultValue == null) ? 0 : defaultValue.hashCode()); return result; } @@ -57,7 +94,7 @@ public boolean equals(Object obj) { } RecordField other = (RecordField) obj; - return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName()); + return dataType.equals(other.getDataType()) && fieldName.equals(other.getFieldName()) && aliases.equals(other.getAliases()) && Objects.equals(defaultValue, other.defaultValue); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java index 8215166e2fba..367f2b0b53a1 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/RecordSchema.java @@ -66,6 +66,12 @@ public interface RecordSchema { */ Optional getSchemaFormat(); + /** + * @param fieldName the name of the field + * @return an Optional RecordField for the field with the given name + */ + Optional getField(String fieldName); + /** * @return the SchemaIdentifier, which provides various attributes for identifying a schema */ diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index 798946ca898d..05b3157d2b04 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -299,7 +299,7 @@ public static String toString(final Object value, final String dateFormat, final } public static boolean isStringTypeCompatible(final Object value) { - return value != null && (value instanceof String || value instanceof java.util.Date); + return value != null; } public static java.sql.Date toDate(final Object value, final String format, final String fieldName) { diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java new file mode 100644 index 000000000000..5a61275ae15c --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/TestSimpleRecordSchema.java @@ -0,0 +1,79 @@ +/* + * 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; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordFieldType; +import org.junit.Assert; +import org.junit.Test; + +public class TestSimpleRecordSchema { + + @Test + public void testPreventsTwoFieldsWithSameAlias() { + final List 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"))); + + try { + new SimpleRecordSchema(fields); + Assert.fail("Was able to create two fields with same alias"); + } catch (final IllegalArgumentException expected) { + } + } + + @Test + public void testPreventsTwoFieldsWithSameName() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar"))); + fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType())); + + try { + new SimpleRecordSchema(fields); + Assert.fail("Was able to create two fields with same name"); + } catch (final IllegalArgumentException expected) { + } + } + + @Test + public void testPreventsTwoFieldsWithConflictingNamesAliases() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("foo", "bar"))); + fields.add(new RecordField("bar", RecordFieldType.STRING.getDataType())); + + try { + new SimpleRecordSchema(fields); + Assert.fail("Was able to create two fields with conflicting names/aliases"); + } catch (final IllegalArgumentException expected) { + } + } + + private Set set(final String... values) { + final Set set = new HashSet<>(); + for (final String value : values) { + set.add(value); + } + return set; + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java new file mode 100644 index 000000000000..82e20a6143a0 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/test/java/org/apache/nifi/serialization/record/TestMapRecord.java @@ -0,0 +1,188 @@ +/* + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.nifi.serialization.SimpleRecordSchema; +import org.junit.Assert; +import org.junit.Test; + +public class TestMapRecord { + + @Test + public void testDefaultValue() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("noDefault", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("defaultOfHello", RecordFieldType.STRING.getDataType(), "hello")); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + final Record record = new MapRecord(schema, values); + + assertNull(record.getValue("noDefault")); + assertEquals("hello", record.getValue("defaultOfHello")); + } + + @Test + public void testDefaultValueInGivenField() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("noDefault", RecordFieldType.STRING.getDataType())); + fields.add(new RecordField("defaultOfHello", RecordFieldType.STRING.getDataType(), "hello")); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + final Record record = new MapRecord(schema, values); + + assertNull(record.getValue("noDefault")); + assertEquals("hello", record.getValue("defaultOfHello")); + + final RecordField newField = new RecordField("noDefault", RecordFieldType.STRING.getDataType(), "new"); + assertEquals("new", record.getValue(newField)); + } + + @Test + public void testIllegalDefaultValue() { + new RecordField("hello", RecordFieldType.STRING.getDataType(), 84); + new RecordField("hello", RecordFieldType.STRING.getDataType(), (Object) null); + new RecordField("hello", RecordFieldType.INT.getDataType(), 84); + new RecordField("hello", RecordFieldType.INT.getDataType(), (Object) null); + + try { + new RecordField("hello", RecordFieldType.INT.getDataType(), "foo"); + Assert.fail("Was able to set a default value of \"foo\" for INT type"); + } catch (final IllegalArgumentException expected) { + // expected + } + } + + private Set set(final String... values) { + final Set set = new HashSet<>(); + for (final String value : values) { + set.add(value); + } + return set; + } + + @Test + public void testAliasOneValue() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + values.put("bar", 1); + + final Record record = new MapRecord(schema, values); + assertEquals(1, record.getValue("foo")); + assertEquals(1, record.getValue("bar")); + assertEquals(1, record.getValue("baz")); + } + + @Test + public void testAliasConflictingValues() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + values.put("bar", 1); + values.put("foo", null); + + final Record record = new MapRecord(schema, values); + assertEquals(1, record.getValue("foo")); + assertEquals(1, record.getValue("bar")); + assertEquals(1, record.getValue("baz")); + } + + @Test + public void testAliasConflictingAliasValues() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + values.put("baz", 1); + values.put("bar", 33); + + final Record record = new MapRecord(schema, values); + assertEquals(33, record.getValue("foo")); + assertEquals(33, record.getValue("bar")); + assertEquals(33, record.getValue("baz")); + } + + @Test + public void testAliasInGivenField() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), null, set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + values.put("bar", 33); + + final Record record = new MapRecord(schema, values); + assertEquals(33, record.getValue("foo")); + assertEquals(33, record.getValue("bar")); + assertEquals(33, record.getValue("baz")); + + final RecordField noAlias = new RecordField("hello", RecordFieldType.STRING.getDataType()); + assertNull(record.getValue(noAlias)); + + final RecordField withAlias = new RecordField("hello", RecordFieldType.STRING.getDataType(), null, set("baz")); + assertEquals(33, record.getValue(withAlias)); + assertEquals("33", record.getAsString(withAlias, withAlias.getDataType().getFormat())); + } + + + @Test + public void testDefaultValueWithAliasValue() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), "hello", set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + values.put("baz", 1); + values.put("bar", 33); + + final Record record = new MapRecord(schema, values); + assertEquals(33, record.getValue("foo")); + assertEquals(33, record.getValue("bar")); + assertEquals(33, record.getValue("baz")); + } + + @Test + public void testDefaultValueWithAliasesDefined() { + final List fields = new ArrayList<>(); + fields.add(new RecordField("foo", RecordFieldType.STRING.getDataType(), "hello", set("bar", "baz"))); + + final RecordSchema schema = new SimpleRecordSchema(fields); + final Map values = new HashMap<>(); + final Record record = new MapRecord(schema, values); + assertEquals("hello", record.getValue("foo")); + assertEquals("hello", record.getValue("bar")); + assertEquals("hello", record.getValue("baz")); + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java index 50fcaf7dfd30..621ec742fa3e 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordReader.java @@ -66,9 +66,18 @@ public Record nextRecord() throws IOException, MalformedRecordException { private Map convertAvroRecordToMap(final GenericRecord avroRecord, final RecordSchema recordSchema) { final Map values = new HashMap<>(recordSchema.getFieldCount()); - for (final String fieldName : recordSchema.getFieldNames()) { - final Object value = avroRecord.get(fieldName); + for (final RecordField recordField : recordSchema.getFields()) { + Object value = avroRecord.get(recordField.getFieldName()); + if (value == null) { + for (final String alias : recordField.getAliases()) { + value = avroRecord.get(alias); + if (value != null) { + break; + } + } + } + final String fieldName = recordField.getFieldName(); final Field avroField = avroRecord.getSchema().getField(fieldName); if (avroField == null) { values.put(fieldName, null); @@ -78,7 +87,7 @@ private Map convertAvroRecordToMap(final GenericRecord avroRecor final Schema fieldSchema = avroField.schema(); final Object rawValue = normalizeValue(value, fieldSchema); - final DataType desiredType = recordSchema.getDataType(fieldName).get(); + final DataType desiredType = recordField.getDataType(); final Object coercedValue = DataTypeUtils.convertType(rawValue, desiredType, fieldName); values.put(fieldName, coercedValue); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java index db5f29dfb4d4..b65026a2aa13 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java @@ -117,7 +117,7 @@ public static DataType determineDataType(final Schema avroSchema) { final String fieldName = field.name(); final Schema fieldSchema = field.schema(); final DataType fieldType = determineDataType(fieldSchema); - recordFields.add(new RecordField(fieldName, fieldType)); + recordFields.add(new RecordField(fieldName, fieldType, field.defaultVal(), field.aliases())); } final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY); @@ -156,7 +156,8 @@ public static RecordSchema createSchema(final Schema avroSchema) { for (final Field field : avroSchema.getFields()) { final String fieldName = field.name(); final DataType dataType = AvroTypeUtil.determineDataType(field.schema()); - recordFields.add(new RecordField(fieldName, dataType)); + + recordFields.add(new RecordField(fieldName, dataType, field.defaultVal(), field.aliases())); } final RecordSchema recordSchema = new SimpleRecordSchema(recordFields, avroSchema.toString(), AVRO_SCHEMA_FORMAT, SchemaIdentifier.EMPTY); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java index 1c0b287685d6..55f796aba6db 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/WriteAvroResult.java @@ -42,6 +42,7 @@ import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.WriteResult; 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.util.DataTypeUtils; @@ -62,8 +63,9 @@ protected GenericRecord createAvroRecord(final Record record, final Schema avroS final GenericRecord rec = new GenericData.Record(avroSchema); final RecordSchema recordSchema = record.getSchema(); - for (final String fieldName : recordSchema.getFieldNames()) { - final Object rawValue = record.getValue(fieldName); + for (final RecordField recordField : recordSchema.getFields()) { + final Object rawValue = record.getValue(recordField); + final String fieldName = recordField.getFieldName(); final Field field = avroSchema.getField(fieldName); if (field == null) { @@ -138,10 +140,10 @@ protected Object convertToAvroObject(final Object rawValue, final Schema fieldSc if (rawValue instanceof Record) { final Record recordValue = (Record) rawValue; final Map map = new HashMap<>(); - for (final String recordFieldName : recordValue.getSchema().getFieldNames()) { - final Object v = recordValue.getValue(recordFieldName); + for (final RecordField recordField : recordValue.getSchema().getFields()) { + final Object v = recordValue.getValue(recordField); if (v != null) { - map.put(recordFieldName, v); + map.put(recordField.getFieldName(), v); } } @@ -153,8 +155,9 @@ protected Object convertToAvroObject(final Object rawValue, final Schema fieldSc final GenericData.Record avroRecord = new GenericData.Record(fieldSchema); final Record record = (Record) rawValue; - for (final String recordFieldName : record.getSchema().getFieldNames()) { - final Object recordFieldValue = record.getValue(recordFieldName); + for (final RecordField recordField : record.getSchema().getFields()) { + final Object recordFieldValue = record.getValue(recordField); + final String recordFieldName = recordField.getFieldName(); final Field field = fieldSchema.getField(recordFieldName); if (field == null) { diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java index fed52f5a761a..fb34f8fcc6d7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVReader.java @@ -110,4 +110,9 @@ protected List getSchemaAccessStrategyValues() { allowableValues.add(headerDerivedAllowableValue); return allowableValues; } + + @Override + protected AllowableValue getDefaultSchemaAccessStrategy() { + return headerDerivedAllowableValue; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java index e80057f7418d..241d604f32c5 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordReader.java @@ -34,6 +34,7 @@ import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.util.DataTypeUtils; @@ -65,15 +66,27 @@ public Record nextRecord() throws IOException, MalformedRecordException { for (final CSVRecord csvRecord : csvParser) { final Map rowValues = new HashMap<>(schema.getFieldCount()); - for (final String fieldName : schema.getFieldNames()) { - final String rawValue = csvRecord.get(fieldName); + for (final RecordField recordField : schema.getFields()) { + String rawValue = csvRecord.get(recordField.getFieldName()); + if (rawValue == null) { + for (final String alias : recordField.getAliases()) { + rawValue = csvRecord.get(alias); + if (rawValue != null) { + break; + } + } + } + + final String fieldName = recordField.getFieldName(); if (rawValue == null) { rowValues.put(fieldName, null); continue; } - final Object converted = convert(rawValue, schema.getDataType(fieldName).orElse(null), fieldName); - rowValues.put(fieldName, converted); + final Object converted = convert(rawValue, recordField.getDataType(), fieldName); + if (converted != null) { + rowValues.put(fieldName, converted); + } } return new MapRecord(schema, rowValues); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java index ec43dce40cde..7c53ace9e720 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/WriteCSVResult.java @@ -21,7 +21,6 @@ import java.io.OutputStream; import java.io.OutputStreamWriter; import java.util.Collections; -import java.util.Optional; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVPrinter; @@ -30,6 +29,7 @@ import org.apache.nifi.serialization.WriteResult; 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.RecordSchema; import org.apache.nifi.serialization.record.RecordSet; import org.apache.nifi.stream.io.NonCloseableOutputStream; @@ -54,13 +54,8 @@ public WriteCSVResult(final CSVFormat csvFormat, final RecordSchema recordSchema this.includeHeaderLine = includeHeaderLine; } - private String getFormat(final Record record, final String fieldName) { - final Optional dataTypeOption = record.getSchema().getDataType(fieldName); - if (!dataTypeOption.isPresent()) { - return null; - } - - final DataType dataType = dataTypeOption.get(); + private String getFormat(final Record record, final RecordField field) { + final DataType dataType = field.getDataType(); switch (dataType.getFieldType()) { case DATE: return dateFormat == null ? dataType.getFormat() : dateFormat; @@ -91,8 +86,8 @@ public WriteResult write(final RecordSet rs, final OutputStream rawOut) throws I while ((record = rs.next()) != null) { final Object[] colVals = new Object[recordSchema.getFieldCount()]; int i = 0; - for (final String fieldName : recordSchema.getFieldNames()) { - colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName)); + for (final RecordField recordField : recordSchema.getFields()) { + colVals[i++] = record.getAsString(recordField, getFormat(record, recordField)); } printer.printRecord(colVals); @@ -117,8 +112,8 @@ public WriteResult write(final Record record, final OutputStream rawOut) throws final RecordSchema schema = record.getSchema(); final Object[] colVals = new Object[schema.getFieldCount()]; int i = 0; - for (final String fieldName : schema.getFieldNames()) { - colVals[i++] = record.getAsString(fieldName, getFormat(record, fieldName)); + for (final RecordField recordField : schema.getFields()) { + colVals[i++] = record.getAsString(recordField, getFormat(record, recordField)); } printer.printRecord(colVals); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java index c00866535e61..6d3a16d8a3a0 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java @@ -32,6 +32,7 @@ import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; 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.util.DataTypeUtils; @@ -111,14 +112,24 @@ public Record nextRecord() throws IOException, MalformedRecordException { final List fieldTypes = schema.getDataTypes(); final Map values = new HashMap<>(fieldTypes.size()); - for (final String fieldName : schema.getFieldNames()) { - final Object value = valueMap.get(fieldName); + for (final RecordField field : schema.getFields()) { + Object value = valueMap.get(field.getFieldName()); + if (value == null) { + for (final String alias : field.getAliases()) { + value = valueMap.get(alias); + if (value != null) { + break; + } + } + } + + final String fieldName = field.getFieldName(); if (value == null) { values.put(fieldName, null); continue; } - final DataType fieldType = schema.getDataType(fieldName).orElse(null); + final DataType fieldType = field.getDataType(); final Object converted = convert(fieldType, value.toString(), fieldName); values.put(fieldName, converted); } 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/JsonPathRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java index 28340452694f..8675e0e2b6f5 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonPathRowRecordReader.java @@ -30,6 +30,7 @@ import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.MapRecord; 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.type.ArrayDataType; @@ -106,7 +107,10 @@ protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSc value = null; } - value = convert(value, desiredType, fieldName); + final Optional field = schema.getField(fieldName); + final Object defaultValue = field.isPresent() ? field.get().getDefaultValue() : null; + + value = convert(value, desiredType, fieldName, defaultValue); values.put(fieldName, value); } @@ -115,9 +119,9 @@ protected Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSc @SuppressWarnings("unchecked") - protected Object convert(final Object value, final DataType dataType, final String fieldName) { + protected Object convert(final Object value, final DataType dataType, final String fieldName, final Object defaultValue) { if (value == null) { - return null; + return defaultValue; } if (value instanceof List) { @@ -131,7 +135,7 @@ protected Object convert(final Object value, final DataType dataType, final Stri final Object[] coercedValues = new Object[list.size()]; int i = 0; for (final Object rawValue : list) { - coercedValues[i++] = convert(rawValue, arrayType.getElementType(), fieldName); + coercedValues[i++] = convert(rawValue, arrayType.getElementType(), fieldName, null); } return coercedValues; } @@ -147,7 +151,10 @@ protected Object convert(final Object value, final DataType dataType, final Stri final String key = entry.getKey(); final Optional desiredTypeOption = childSchema.getDataType(key); if (desiredTypeOption.isPresent()) { - final Object coercedValue = convert(entry.getValue(), desiredTypeOption.get(), fieldName + "." + key); + final Optional field = childSchema.getField(key); + final Object defaultFieldValue = field.isPresent() ? field.get().getDefaultValue() : null; + + final Object coercedValue = convert(entry.getValue(), desiredTypeOption.get(), fieldName + "." + key, defaultFieldValue); coercedValues.put(key, coercedValue); } } 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/JsonTreeRowRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java index e396e8eea3fe..301b724ce400 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/JsonTreeRowRecordReader.java @@ -73,7 +73,16 @@ private Record convertJsonNodeToRecord(final JsonNode jsonNode, final RecordSche for (int i = 0; i < schema.getFieldCount(); i++) { final RecordField field = schema.getField(i); final String fieldName = field.getFieldName(); - final JsonNode fieldNode = jsonNode.get(fieldName); + + JsonNode fieldNode = jsonNode.get(fieldName); + if (fieldNode == null) { + for (final String alias : field.getAliases()) { + fieldNode = jsonNode.get(alias); + if (fieldNode != null) { + break; + } + } + } final DataType desiredType = field.getDataType(); final Object value = convertField(fieldNode, fieldNamePrefix + fieldName, desiredType); 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/WriteJsonResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java index 3c9517a55d6b..943e1d51f5e8 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/json/WriteJsonResult.java @@ -30,6 +30,7 @@ import org.apache.nifi.serialization.WriteResult; 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 org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.RecordSet; @@ -114,8 +115,9 @@ private void writeRecord(final Record record, final RecordSchema writeSchema, fi try { startTask.apply(generator); for (int i = 0; i < writeSchema.getFieldCount(); i++) { - final String fieldName = writeSchema.getField(i).getFieldName(); - final Object value = record.getValue(fieldName); + final RecordField field = writeSchema.getField(i); + final String fieldName = field.getFieldName(); + final Object value = record.getValue(field); if (value == null) { generator.writeNullField(fieldName); continue; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java index 3ce2cb736f17..af2c16049b46 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java @@ -44,6 +44,9 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content throw new SchemaNotFoundException("Could not read first 13 bytes from stream", ioe); } + // This encoding follows the pattern that is provided for serializing data by the Hortonworks Schema Registry serializer + // as it is provided at: + // https://github.com/hortonworks/registry/blob/master/schema-registry/serdes/src/main/java/com/hortonworks/registries/schemaregistry/serdes/avro/AvroSnapshotSerializer.java final ByteBuffer bb = ByteBuffer.wrap(buffer); final int protocolVersion = bb.get(); if (protocolVersion != 1) { diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java index f7019e719572..978e335720a8 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java @@ -44,6 +44,9 @@ public void writeHeader(final RecordSchema schema, final OutputStream out) throw throw new IOException("Cannot write Encoded Schema Reference for Schema because the Schema Version is not known"); } + // This decoding follows the pattern that is provided for serializing data by the Hortonworks Schema Registry serializer + // as it is provided at: + // https://github.com/hortonworks/registry/blob/master/schema-registry/serdes/src/main/java/com/hortonworks/registries/schemaregistry/serdes/avro/AvroSnapshotSerializer.java final ByteBuffer bb = ByteBuffer.allocate(13); bb.put((byte) LATEST_PROTOCOL_VERSION); bb.putLong(identifierOption.getAsLong()); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java index 781f41fc72e4..7fdc7a42a5cd 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/text/FreeFormTextWriter.java @@ -20,8 +20,10 @@ import java.io.IOException; import java.io.OutputStream; import java.nio.charset.Charset; +import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.nifi.components.PropertyValue; @@ -29,6 +31,7 @@ import org.apache.nifi.serialization.RecordSetWriter; import org.apache.nifi.serialization.WriteResult; import org.apache.nifi.serialization.record.Record; +import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.RecordSet; @@ -48,7 +51,7 @@ public WriteResult write(final RecordSet recordSet, final OutputStream out) thro try { final RecordSchema schema = recordSet.getSchema(); - final String[] colNames = getColumnNames(schema); + final List colNames = getColumnNames(schema); Record record; while ((record = recordSet.next()) != null) { @@ -62,11 +65,13 @@ public WriteResult write(final RecordSet recordSet, final OutputStream out) thro return WriteResult.of(count, Collections.emptyMap()); } - private String[] getColumnNames(final RecordSchema schema) { - final int numCols = schema.getFieldCount(); - final String[] columnNames = new String[numCols]; - for (int i = 0; i < numCols; i++) { - columnNames[i] = schema.getField(i).getFieldName(); + private List getColumnNames(final RecordSchema schema) { + final List columnNames = new ArrayList<>(); + for (final RecordField field : schema.getFields()) { + columnNames.add(field.getFieldName()); + for (final String alias : field.getAliases()) { + columnNames.add(alias); + } } return columnNames; @@ -78,11 +83,11 @@ public WriteResult write(final Record record, final OutputStream out) throws IOE return WriteResult.of(1, Collections.emptyMap()); } - private void write(final Record record, final OutputStream out, final String[] columnNames) throws IOException { - final int numCols = columnNames.length; + private void write(final Record record, final OutputStream out, final List columnNames) throws IOException { + final int numCols = columnNames.size(); final Map values = new HashMap<>(numCols); for (int i = 0; i < numCols; i++) { - final String columnName = columnNames[i]; + final String columnName = columnNames.get(i); final String columnValue = record.getAsString(columnName); values.put(columnName, columnValue); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java index f771a590a249..409ede2f12a4 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/avro/TestWriteAvroResult.java @@ -110,7 +110,6 @@ public void testLogicalTypes() throws IOException, ParseException { @Test public void testDataTypes() throws IOException { - // TODO: Test Enums final Schema schema = new Schema.Parser().parse(new File("src/test/resources/avro/datatypes.avsc")); final WriteAvroResult writer = createWriter(schema); @@ -124,7 +123,7 @@ public void testDataTypes() throws IOException { fields.add(new RecordField("double", RecordFieldType.DOUBLE.getDataType())); fields.add(new RecordField("float", RecordFieldType.FLOAT.getDataType())); fields.add(new RecordField("boolean", RecordFieldType.BOOLEAN.getDataType())); - fields.add(new RecordField("bytes", RecordFieldType.ARRAY.getChoiceDataType(Collections.singletonList(RecordFieldType.BYTE.getDataType())))); + fields.add(new RecordField("bytes", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.BYTE.getDataType()))); fields.add(new RecordField("nullOrLong", RecordFieldType.LONG.getDataType())); fields.add(new RecordField("array", RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.INT.getDataType()))); fields.add(new RecordField("record", RecordFieldType.RECORD.getRecordDataType(subRecordSchema))); From b8372212a890d20fe14adc355ada2c91a6ad49a8 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 21 Apr 2017 13:07:56 -0400 Subject: [PATCH 5/8] NIFI-3682: Added HWX Schema Reference Attribute strategies for reading/writing schemas --- .../apache/nifi/avro/AvroRecordSetWriter.java | 2 +- .../apache/nifi/csv/CSVRecordSetWriter.java | 2 +- .../apache/nifi/json/JsonRecordSetWriter.java | 2 +- ...worksAttributeSchemaReferenceStrategy.java | 101 ++++++++++++++++++ ...onworksAttributeSchemaReferenceWriter.java | 61 +++++++++++ ...rtonworksEncodedSchemaReferenceWriter.java | 29 +++-- .../schema/access/SchemaAccessWriter.java | 8 ++ .../schema/access/SchemaNameAsAttribute.java | 8 ++ .../schema/access/SchemaTextAsAttribute.java | 20 ++-- .../SchemaRegistryRecordSetWriter.java | 15 ++- .../serialization/SchemaRegistryService.java | 12 ++- 11 files changed, 233 insertions(+), 27 deletions(-) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java index fc1482b9bec2..84b749f4eef3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java @@ -72,7 +72,7 @@ public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile fl if (AVRO_EMBEDDED.getValue().equals(strategyValue)) { return new WriteAvroResultWithSchema(avroSchema); } else { - return new WriteAvroResultWithExternalSchema(avroSchema, recordSchema, getSchemaAccessWriter()); + return new WriteAvroResultWithExternalSchema(avroSchema, recordSchema, getSchemaAccessWriter(recordSchema)); } } catch (final SchemaNotFoundException e) { throw new ProcessException("Could not determine the Avro Schema to use for writing the content", e); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java index f1d7fecde037..95c86e7c1166 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVRecordSetWriter.java @@ -70,6 +70,6 @@ public void storeCsvFormat(final ConfigurationContext context) { @Override public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream in) throws SchemaNotFoundException, IOException { final RecordSchema schema = getSchema(flowFile, in); - return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(), getDateFormat(), getTimeFormat(), getTimestampFormat(), includeHeader); + return new WriteCSVResult(csvFormat, schema, getSchemaAccessWriter(schema), getDateFormat(), getTimeFormat(), getTimestampFormat(), includeHeader); } } 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 9e5454724aec..e6b5c029b2ef 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 @@ -66,7 +66,7 @@ public void onEnabled(final ConfigurationContext context) { @Override public RecordSetWriter createWriter(final ComponentLog logger, final FlowFile flowFile, final InputStream flowFileContent) throws SchemaNotFoundException, IOException { final RecordSchema schema = getSchema(flowFile, flowFileContent); - return new WriteJsonResult(logger, schema, getSchemaAccessWriter(), prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat()); + return new WriteJsonResult(logger, schema, getSchemaAccessWriter(schema), prettyPrint, getDateFormat(), getTimeFormat(), getTimestampFormat()); } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java new file mode 100644 index 000000000000..f3c9b6b17a9b --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java @@ -0,0 +1,101 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.InputStream; + +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.schemaregistry.services.SchemaRegistry; +import org.apache.nifi.serialization.record.RecordSchema; + +public class HortonworksAttributeSchemaReferenceStrategy implements SchemaAccessStrategy { + public static final String SCHEMA_ID_ATTRIBUTE = "schema.identifier"; + public static final String SCHEMA_VERSION_ATTRIBUTE = "schema.version"; + public static final String SCHEMA_PROTOCOL_VERSION_ATTRIBUTE = "schema.protocol.version"; + + private final SchemaRegistry schemaRegistry; + + + public HortonworksAttributeSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) { + this.schemaRegistry = schemaRegistry; + } + + @Override + public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException, IOException { + final String schemaIdentifier = flowFile.getAttribute(SCHEMA_ID_ATTRIBUTE); + final String schemaVersion = flowFile.getAttribute(SCHEMA_VERSION_ATTRIBUTE); + final String schemaProtocol = flowFile.getAttribute(SCHEMA_PROTOCOL_VERSION_ATTRIBUTE); + if (schemaIdentifier == null || schemaVersion == null || schemaProtocol == null) { + throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because it is missing one of the following three required attributes: " + + SCHEMA_ID_ATTRIBUTE + ", " + SCHEMA_VERSION_ATTRIBUTE + ", " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE); + } + + if (!isNumber(schemaProtocol)) { + throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE + " has a value of '" + + schemaProtocol + "', which is not a valid Protocol Version number"); + } + + final int protocol = Integer.parseInt(schemaProtocol); + if (protocol != 1) { + throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_PROTOCOL_VERSION_ATTRIBUTE + " has a value of '" + + schemaProtocol + "', which is not a valid Protocol Version number. Expected Protocol Version to be 1."); + } + + if (!isNumber(schemaIdentifier)) { + throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_ID_ATTRIBUTE + " has a value of '" + + schemaProtocol + "', which is not a valid Schema Identifier number"); + } + + if (!isNumber(schemaVersion)) { + throw new SchemaNotFoundException("Could not determine Schema for " + flowFile + " because the " + SCHEMA_VERSION_ATTRIBUTE + " has a value of '" + + schemaProtocol + "', which is not a valid Schema Version number"); + } + + final long schemaId = Long.parseLong(schemaIdentifier); + final int version = Integer.parseInt(schemaVersion); + + final RecordSchema schema = schemaRegistry.retrieveSchema(schemaId, version); + if (schema == null) { + throw new SchemaNotFoundException("Could not find a Schema in the Schema Registry with Schema Identifier '" + schemaId + "' and Version '" + version + "'"); + } + + return schema; + } + + private static boolean isNumber(final String value) { + if (value == null) { + return false; + } + + final String trimmed = value.trim(); + if (value.isEmpty()) { + return false; + } + + for (int i = 0; i < trimmed.length(); i++) { + final char c = value.charAt(i); + if (c > '9' || c < '0') { + return false; + } + } + + return true; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java new file mode 100644 index 000000000000..76f2488ca8c9 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java @@ -0,0 +1,61 @@ +/* + * 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.schema.access; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.HashMap; +import java.util.Map; + +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; + +public class HortonworksAttributeSchemaReferenceWriter implements SchemaAccessWriter { + private static final int LATEST_PROTOCOL_VERSION = 1; + + @Override + public void writeHeader(RecordSchema schema, OutputStream out) throws IOException { + } + + @Override + public Map getAttributes(final RecordSchema schema) { + final Map attributes = new HashMap<>(4); + final SchemaIdentifier id = schema.getIdentifier(); + + final long schemaId = id.getIdentifier().getAsLong(); + final int schemaVersion = id.getVersion().getAsInt(); + + attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_ID_ATTRIBUTE, String.valueOf(schemaId)); + attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_VERSION_ATTRIBUTE, String.valueOf(schemaVersion)); + attributes.put(HortonworksAttributeSchemaReferenceStrategy.SCHEMA_PROTOCOL_VERSION_ATTRIBUTE, String.valueOf(LATEST_PROTOCOL_VERSION)); + + return attributes; + } + + @Override + public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException { + final SchemaIdentifier id = schema.getIdentifier(); + if (!id.getIdentifier().isPresent()) { + throw new SchemaNotFoundException("Cannot write Schema Reference as Attributes because it does not contain a Schema Identifier"); + } + if (!id.getVersion().isPresent()) { + throw new SchemaNotFoundException("Cannot write Schema Reference as Attributes because it does not contain a Schema Version"); + } + } + +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java index 978e335720a8..81071bf5f6fc 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java @@ -34,23 +34,16 @@ public class HortonworksEncodedSchemaReferenceWriter implements SchemaAccessWrit @Override public void writeHeader(final RecordSchema schema, final OutputStream out) throws IOException { final SchemaIdentifier identifier = schema.getIdentifier(); - final OptionalLong identifierOption = identifier.getIdentifier(); - if (!identifierOption.isPresent()) { - throw new IOException("Cannot write Encoded Schema Reference for Schema because the Schema Identifier is not known"); - } - - final OptionalInt versionOption = identifier.getVersion(); - if (!versionOption.isPresent()) { - throw new IOException("Cannot write Encoded Schema Reference for Schema because the Schema Version is not known"); - } + final long id = identifier.getIdentifier().getAsLong(); + final int version = identifier.getVersion().getAsInt(); // This decoding follows the pattern that is provided for serializing data by the Hortonworks Schema Registry serializer // as it is provided at: // https://github.com/hortonworks/registry/blob/master/schema-registry/serdes/src/main/java/com/hortonworks/registries/schemaregistry/serdes/avro/AvroSnapshotSerializer.java final ByteBuffer bb = ByteBuffer.allocate(13); bb.put((byte) LATEST_PROTOCOL_VERSION); - bb.putLong(identifierOption.getAsLong()); - bb.putInt(versionOption.getAsInt()); + bb.putLong(id); + bb.putInt(version); out.write(bb.array()); } @@ -60,4 +53,18 @@ public Map getAttributes(final RecordSchema schema) { return Collections.emptyMap(); } + @Override + public void validateSchema(RecordSchema schema) throws SchemaNotFoundException { + final SchemaIdentifier identifier = schema.getIdentifier(); + final OptionalLong identifierOption = identifier.getIdentifier(); + if (!identifierOption.isPresent()) { + throw new SchemaNotFoundException("Cannot write Encoded Schema Reference because the Schema Identifier is not known"); + } + + final OptionalInt versionOption = identifier.getVersion(); + if (!versionOption.isPresent()) { + throw new SchemaNotFoundException("Cannot write Encoded Schema Reference because the Schema Version is not known"); + } + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java index 4a5d76394ad4..078118271793 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java @@ -44,4 +44,12 @@ public interface SchemaAccessWriter { */ Map getAttributes(RecordSchema schema); + /** + * Ensures that the given schema can be written by this SchemaAccessWriter or throws SchemaNotFoundException if + * the schema does not contain sufficient information to be written + * + * @param schema the schema to validate + * @throws SchemaNotFoundException if the schema does not contain sufficient information to be written + */ + void validateSchema(RecordSchema schema) throws SchemaNotFoundException; } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java index c4bb86364905..20e6d50b7d2d 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java @@ -43,4 +43,12 @@ public Map getAttributes(final RecordSchema schema) { return Collections.emptyMap(); } + @Override + public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException { + final SchemaIdentifier schemaId = schema.getIdentifier(); + if (!schemaId.getName().isPresent()) { + throw new SchemaNotFoundException("Cannot write Schema Name As Attribute because the Schema Name is not known"); + } + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java index ba98fb4524bd..81766ea3d801 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java @@ -33,14 +33,20 @@ public void writeHeader(final RecordSchema schema, final OutputStream out) { @Override public Map getAttributes(final RecordSchema schema) { final Optional textFormatOption = schema.getSchemaFormat(); - if (textFormatOption.isPresent()) { - final Optional textOption = schema.getSchemaText(); - if (textOption.isPresent()) { - return Collections.singletonMap(textFormatOption.get() + ".schema", textOption.get()); - } + final Optional textOption = schema.getSchemaText(); + return Collections.singletonMap(textFormatOption.get() + ".schema", textOption.get()); + } + + @Override + public void validateSchema(final RecordSchema schema) throws SchemaNotFoundException { + final Optional textFormatOption = schema.getSchemaFormat(); + if (!textFormatOption.isPresent()) { + throw new SchemaNotFoundException("Cannot write Schema Text as Attribute because the Schema's Text Format is not present"); } - return Collections.emptyMap(); + final Optional textOption = schema.getSchemaText(); + if (!textOption.isPresent()) { + throw new SchemaNotFoundException("Cannot write Schema Text as Attribute because the Schema's Text is not present"); + } } - } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java index cd57e8b49c4a..5301b71d37a3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java @@ -26,10 +26,13 @@ import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceWriter; import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceWriter; import org.apache.nifi.schema.access.SchemaAccessWriter; import org.apache.nifi.schema.access.SchemaNameAsAttribute; +import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schema.access.SchemaTextAsAttribute; +import org.apache.nifi.serialization.record.RecordSchema; public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService { @@ -44,11 +47,14 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic + "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. " + "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", + "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."); protected static final PropertyDescriptor SCHEMA_WRITE_STRATEGY = new PropertyDescriptor.Builder() .name("Schema Write Strategy") .description("Specifies how the schema for a Record should be added to the data.") - .allowableValues(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_CONTENT_ENCODED_SCHEMA) + .allowableValues(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA) .defaultValue(AVRO_SCHEMA_ATTRIBUTE.getValue()) .required(true) .build(); @@ -57,7 +63,7 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic private volatile ConfigurationContext configurationContext; private volatile SchemaAccessWriter schemaAccessWriter; - private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_CONTENT_ENCODED_SCHEMA)); + private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_ATTRIBUTE, AVRO_SCHEMA_ATTRIBUTE, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)); @Override @@ -91,7 +97,8 @@ protected ConfigurationContext getConfigurationContext() { return configurationContext; } - protected SchemaAccessWriter getSchemaAccessWriter() { + protected SchemaAccessWriter getSchemaAccessWriter(final RecordSchema schema) throws SchemaNotFoundException { + schemaAccessWriter.validateSchema(schema); return schemaAccessWriter; } @@ -106,6 +113,8 @@ protected SchemaAccessWriter getSchemaWriteStrategy(final String allowableValue) return new SchemaTextAsAttribute(); } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) { return new HortonworksEncodedSchemaReferenceWriter(); + } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) { + return new HortonworksAttributeSchemaReferenceWriter(); } return null; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java index f4a6b7a74c12..ed4d97dc2e4e 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java @@ -36,6 +36,7 @@ import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.schema.access.AvroSchemaTextStrategy; +import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceStrategy; import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceStrategy; import org.apache.nifi.schema.access.SchemaAccessStrategy; import org.apache.nifi.schema.access.SchemaNamePropertyStrategy; @@ -53,6 +54,8 @@ public abstract class SchemaRegistryService extends AbstractControllerService { static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference", "The content of the FlowFile contains 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."); + static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes", + "The FlowFile contains 3 Attributes that will be used to lookup a Schema from the configured Schema Registry: 'schema.identifier', 'schema.version', and 'schema.protocol.version'"); protected static final PropertyDescriptor SCHEMA_REGISTRY = new PropertyDescriptor.Builder() .name("Schema Registry") @@ -64,7 +67,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService { protected static final PropertyDescriptor SCHEMA_ACCESS_STRATEGY = new PropertyDescriptor.Builder() .name("Schema Access Strategy") .description("Specifies how to obtain the schema that is to be used for interpreting the data.") - .allowableValues(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_CONTENT_ENCODED_SCHEMA) + .allowableValues(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA) .defaultValue(SCHEMA_TEXT_PROPERTY.getValue()) .required(true) .build(); @@ -92,7 +95,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService { private volatile ConfigurationContext configurationContext; private volatile SchemaAccessStrategy schemaAccessStrategy; - private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_CONTENT_ENCODED_SCHEMA)); + private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)); @Override @@ -151,7 +154,8 @@ private String getSchemaAccessStrategyName(final String schemaAccessValue) { } private boolean isSchemaRegistryRequired(final String schemaAccessValue) { - return HWX_CONTENT_ENCODED_SCHEMA.getValue().equalsIgnoreCase(schemaAccessValue) || SCHEMA_NAME_PROPERTY.getValue().equalsIgnoreCase(schemaAccessValue); + return HWX_CONTENT_ENCODED_SCHEMA.getValue().equalsIgnoreCase(schemaAccessValue) || SCHEMA_NAME_PROPERTY.getValue().equalsIgnoreCase(schemaAccessValue) + || HWX_SCHEMA_REF_ATTRIBUTES.getValue().equalsIgnoreCase(schemaAccessValue); } @Override @@ -184,6 +188,8 @@ protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableVal return new AvroSchemaTextStrategy(getConfigurationContext().getProperty(SCHEMA_TEXT)); } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) { return new HortonworksEncodedSchemaReferenceStrategy(schemaRegistry); + } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) { + return new HortonworksAttributeSchemaReferenceStrategy(schemaRegistry); } return null; From dbc70a330a637960b168089a6917fd676a2d12c3 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Fri, 21 Apr 2017 17:00:48 -0400 Subject: [PATCH 6/8] NIFI-3682: Added additional validation for Record Writers so that the configured Schema Access Strategy and Schema Write Strategy must align --- .../services/AvroSchemaRegistry.java | 10 ++++- .../nifi/schema/access/SchemaField.java | 37 ++++++++++++++++ .../apache/nifi/avro/AvroRecordSetWriter.java | 15 +++++++ .../nifi/csv/CSVHeaderSchemaStrategy.java | 8 ++++ .../java/org/apache/nifi/grok/GrokReader.java | 10 +++++ .../schema/access/AvroSchemaTextStrategy.java | 8 ++++ ...worksAttributeSchemaReferenceStrategy.java | 15 +++++++ ...onworksAttributeSchemaReferenceWriter.java | 8 ++++ ...onworksEncodedSchemaReferenceStrategy.java | 14 ++++++ ...rtonworksEncodedSchemaReferenceWriter.java | 8 ++++ .../schema/access/SchemaAccessStrategy.java | 14 ++++++ .../schema/access/SchemaAccessWriter.java | 8 ++++ .../schema/access/SchemaNameAsAttribute.java | 8 ++++ .../access/SchemaNamePropertyStrategy.java | 14 ++++++ .../schema/access/SchemaTextAsAttribute.java | 8 ++++ .../SchemaRegistryRecordSetWriter.java | 43 +++++++++++++++++-- .../serialization/SchemaRegistryService.java | 29 ++++++++++++- .../services/SchemaRegistry.java | 18 +++++++- 18 files changed, 267 insertions(+), 8 deletions(-) create mode 100644 nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java diff --git a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java index 0eca4ea472a0..8fcb0160c97d 100644 --- a/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java +++ b/nifi-nar-bundles/nifi-registry-bundle/nifi-registry-service/src/main/java/org/apache/nifi/schemaregistry/services/AvroSchemaRegistry.java @@ -18,9 +18,11 @@ import java.io.IOException; import java.util.ArrayList; +import java.util.EnumSet; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; import org.apache.avro.LogicalType; @@ -35,6 +37,7 @@ import org.apache.nifi.controller.AbstractControllerService; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.reporting.InitializationException; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.DataType; @@ -48,7 +51,7 @@ + "as a dynamic property where 'name' represents the schema name and 'value' represents the textual " + "representation of the actual schema following the syntax and semantics of Avro's Schema format.") public class AvroSchemaRegistry extends AbstractControllerService implements SchemaRegistry { - + private static final Set schemaFields = EnumSet.of(SchemaField.SCHEMA_NAME, SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT); private final Map schemaNameToSchemaMap; private static final String LOGICAL_TYPE_DATE = "date"; @@ -220,4 +223,9 @@ private DataType determineDataType(final Schema avroSchema) { return null; } + + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java new file mode 100644 index 000000000000..e4dbd92456f9 --- /dev/null +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/schema/access/SchemaField.java @@ -0,0 +1,37 @@ +/* + * 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.schema.access; + +public enum SchemaField { + SCHEMA_TEXT("Schema Text"), + SCHEMA_TEXT_FORMAT("Schema Text Format"), + SCHEMA_NAME("Schema Name"), + SCHEMA_IDENTIFIER("Schema Identifier"), + SCHEMA_VERSION("Schema Version"); + + private final String description; + + private SchemaField(final String description) { + this.description = description; + } + + @Override + public String toString() { + return description; + } +} diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java index 84b749f4eef3..381e978bc208 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/avro/AvroRecordSetWriter.java @@ -20,16 +20,20 @@ import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; +import java.util.Set; import org.apache.avro.Schema; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schemaregistry.services.SchemaRegistry; import org.apache.nifi.serialization.RecordSetWriter; @@ -40,6 +44,7 @@ @Tags({"avro", "result", "set", "writer", "serializer", "record", "recordset", "row"}) @CapabilityDescription("Writes the contents of a RecordSet in Binary Avro format.") public class AvroRecordSetWriter extends SchemaRegistryRecordSetWriter implements RecordSetWriterFactory { + private static final Set requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT); static final AllowableValue AVRO_EMBEDDED = new AllowableValue("avro-embedded", "Embed Avro Schema", "The FlowFile will have the Avro schema embedded into the content, as is typical with Avro"); @@ -91,4 +96,14 @@ protected List getSchemaWriteStrategyValues() { protected AllowableValue getDefaultSchemaWriteStrategy() { return AVRO_EMBEDDED; } + + @Override + protected Set getRequiredSchemaFields(final ValidationContext validationContext) { + final String writeStrategyValue = validationContext.getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + if (writeStrategyValue.equalsIgnoreCase(AVRO_EMBEDDED.getValue())) { + return requiredSchemaFields; + } + + return super.getRequiredSchemaFields(validationContext); + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java index 9ad3b8cdbbf0..71093dec1742 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/csv/CSVHeaderSchemaStrategy.java @@ -21,7 +21,9 @@ import java.io.InputStreamReader; import java.io.Reader; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; +import java.util.Set; import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVParser; @@ -29,6 +31,7 @@ import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.RecordField; @@ -36,6 +39,7 @@ import org.apache.nifi.serialization.record.RecordSchema; public class CSVHeaderSchemaStrategy implements SchemaAccessStrategy { + private static final Set schemaFields = EnumSet.noneOf(SchemaField.class); @Override public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { @@ -56,4 +60,8 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content } } + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java index 6af92294369b..596f1dd07d7c 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java @@ -22,8 +22,10 @@ import java.io.InputStreamReader; import java.io.Reader; import java.util.ArrayList; +import java.util.EnumSet; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.regex.Matcher; import org.apache.nifi.annotation.documentation.CapabilityDescription; @@ -36,6 +38,7 @@ import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schemaregistry.services.SchemaRegistry; import org.apache.nifi.serialization.RecordReader; @@ -178,10 +181,17 @@ protected AllowableValue getDefaultSchemaAccessStrategy() { protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) { if (allowableValue.equalsIgnoreCase(STRING_FIELDS_FROM_GROK_EXPRESSION.getValue())) { return new SchemaAccessStrategy() { + private final Set schemaFields = EnumSet.noneOf(SchemaField.class); + @Override public RecordSchema getSchema(final FlowFile flowFile, final InputStream contentStream, final ConfigurationContext context) throws SchemaNotFoundException { return recordSchema; } + + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } }; } else { return super.getSchemaAccessStrategy(allowableValue, schemaRegistry); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java index c1676680f81d..27f84e4764f7 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/AvroSchemaTextStrategy.java @@ -18,6 +18,8 @@ package org.apache.nifi.schema.access; import java.io.InputStream; +import java.util.EnumSet; +import java.util.Set; import org.apache.avro.Schema; import org.apache.nifi.avro.AvroTypeUtil; @@ -29,6 +31,8 @@ import org.slf4j.LoggerFactory; public class AvroSchemaTextStrategy implements SchemaAccessStrategy { + private static final Set schemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT); + private static final Logger logger = LoggerFactory.getLogger(AvroSchemaTextStrategy.class); private final PropertyValue schemaTextPropertyValue; @@ -53,4 +57,8 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content } } + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java index f3c9b6b17a9b..4eec14eea4b3 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceStrategy.java @@ -19,6 +19,9 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; @@ -26,6 +29,8 @@ import org.apache.nifi.serialization.record.RecordSchema; public class HortonworksAttributeSchemaReferenceStrategy implements SchemaAccessStrategy { + private final Set schemaFields; + public static final String SCHEMA_ID_ATTRIBUTE = "schema.identifier"; public static final String SCHEMA_VERSION_ATTRIBUTE = "schema.version"; public static final String SCHEMA_PROTOCOL_VERSION_ATTRIBUTE = "schema.protocol.version"; @@ -35,6 +40,11 @@ public class HortonworksAttributeSchemaReferenceStrategy implements SchemaAccess public HortonworksAttributeSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) { this.schemaRegistry = schemaRegistry; + + schemaFields = new HashSet<>(); + schemaFields.add(SchemaField.SCHEMA_IDENTIFIER); + schemaFields.add(SchemaField.SCHEMA_VERSION); + schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields()); } @Override @@ -98,4 +108,9 @@ private static boolean isNumber(final String value) { return true; } + + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java index 76f2488ca8c9..f492ec4875c4 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksAttributeSchemaReferenceWriter.java @@ -19,13 +19,16 @@ import java.io.IOException; import java.io.OutputStream; +import java.util.EnumSet; import java.util.HashMap; import java.util.Map; +import java.util.Set; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.SchemaIdentifier; public class HortonworksAttributeSchemaReferenceWriter implements SchemaAccessWriter { + private static final Set requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_IDENTIFIER, SchemaField.SCHEMA_VERSION); private static final int LATEST_PROTOCOL_VERSION = 1; @Override @@ -58,4 +61,9 @@ public void validateSchema(final RecordSchema schema) throws SchemaNotFoundExcep } } + @Override + public Set getRequiredSchemaFields() { + return requiredSchemaFields; + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java index af2c16049b46..081e97c59314 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceStrategy.java @@ -20,6 +20,9 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; @@ -29,10 +32,17 @@ public class HortonworksEncodedSchemaReferenceStrategy implements SchemaAccessStrategy { private static final int LATEST_PROTOCOL_VERSION = 1; + + private final Set schemaFields; private final SchemaRegistry schemaRegistry; public HortonworksEncodedSchemaReferenceStrategy(final SchemaRegistry schemaRegistry) { this.schemaRegistry = schemaRegistry; + + schemaFields = new HashSet<>(); + schemaFields.add(SchemaField.SCHEMA_IDENTIFIER); + schemaFields.add(SchemaField.SCHEMA_VERSION); + schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields()); } @Override @@ -60,4 +70,8 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content return schemaRegistry.retrieveSchema(schemaId, schemaVersion); } + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java index 81071bf5f6fc..bf6a9ea1235a 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/HortonworksEncodedSchemaReferenceWriter.java @@ -21,14 +21,17 @@ import java.io.OutputStream; import java.nio.ByteBuffer; import java.util.Collections; +import java.util.EnumSet; import java.util.Map; import java.util.OptionalInt; import java.util.OptionalLong; +import java.util.Set; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.SchemaIdentifier; public class HortonworksEncodedSchemaReferenceWriter implements SchemaAccessWriter { + private static final Set requiredSchemaFields = EnumSet.of(SchemaField.SCHEMA_IDENTIFIER, SchemaField.SCHEMA_VERSION); private static final int LATEST_PROTOCOL_VERSION = 1; @Override @@ -67,4 +70,9 @@ public void validateSchema(RecordSchema schema) throws SchemaNotFoundException { } } + @Override + public Set getRequiredSchemaFields() { + return requiredSchemaFields; + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java index d8dd0a9ec8d2..6635e3d391a5 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessStrategy.java @@ -19,11 +19,25 @@ import java.io.IOException; import java.io.InputStream; +import java.util.Set; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.serialization.record.RecordSchema; public interface SchemaAccessStrategy { + /** + * Returns the schema for the given FlowFile using the supplied stream of content and configuration + * + * @param flowFile flowfile + * @param contentStream content of flowfile + * @param context configuration + * @return the RecordSchema for the FlowFile + */ RecordSchema getSchema(FlowFile flowFile, InputStream contentStream, ConfigurationContext context) throws SchemaNotFoundException, IOException; + + /** + * @return the set of all Schema Fields that are supplied by the RecordSchema that is returned from {@link #getSchema(FlowFile, InputStream, ConfigurationContext)}. + */ + Set getSuppliedSchemaFields(); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java index 078118271793..30a995c970a1 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaAccessWriter.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Map; +import java.util.Set; import org.apache.nifi.serialization.record.RecordSchema; @@ -52,4 +53,11 @@ public interface SchemaAccessWriter { * @throws SchemaNotFoundException if the schema does not contain sufficient information to be written */ void validateSchema(RecordSchema schema) throws SchemaNotFoundException; + + /** + * Specifies the set of SchemaField's that are required in order to use this Schema Access Writer + * + * @return the set of SchemaField's that are required in order to use this Schema Access Writer + */ + Set getRequiredSchemaFields(); } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java index 20e6d50b7d2d..54a248df5589 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNameAsAttribute.java @@ -20,13 +20,16 @@ import java.io.IOException; import java.io.OutputStream; import java.util.Collections; +import java.util.EnumSet; import java.util.Map; import java.util.Optional; +import java.util.Set; import org.apache.nifi.serialization.record.RecordSchema; import org.apache.nifi.serialization.record.SchemaIdentifier; public class SchemaNameAsAttribute implements SchemaAccessWriter { + private static final Set schemaFields = EnumSet.of(SchemaField.SCHEMA_NAME); private static final String SCHEMA_NAME_ATTRIBUTE = "schema.name"; @Override @@ -51,4 +54,9 @@ public void validateSchema(final RecordSchema schema) throws SchemaNotFoundExcep } } + @Override + public Set getRequiredSchemaFields() { + return schemaFields; + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java index b39a1176e1df..bc21c1d54787 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaNamePropertyStrategy.java @@ -18,6 +18,9 @@ package org.apache.nifi.schema.access; import java.io.InputStream; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.apache.nifi.components.PropertyValue; import org.apache.nifi.controller.ConfigurationContext; @@ -26,12 +29,18 @@ import org.apache.nifi.serialization.record.RecordSchema; public class SchemaNamePropertyStrategy implements SchemaAccessStrategy { + private final Set schemaFields; + private final SchemaRegistry schemaRegistry; private final PropertyValue schemaNamePropertyValue; public SchemaNamePropertyStrategy(final SchemaRegistry schemaRegistry, final PropertyValue schemaNamePropertyValue) { this.schemaRegistry = schemaRegistry; this.schemaNamePropertyValue = schemaNamePropertyValue; + + schemaFields = new HashSet<>(); + schemaFields.add(SchemaField.SCHEMA_NAME); + schemaFields.addAll(schemaRegistry == null ? Collections.emptySet() : schemaRegistry.getSuppliedSchemaFields()); } @Override @@ -52,4 +61,9 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content throw new SchemaNotFoundException("Could not retrieve schema with name '" + schemaName + "' from the configured Schema Registry", e); } } + + @Override + public Set getSuppliedSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java index 81766ea3d801..f39bdca2dab9 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/schema/access/SchemaTextAsAttribute.java @@ -19,12 +19,15 @@ import java.io.OutputStream; import java.util.Collections; +import java.util.EnumSet; import java.util.Map; import java.util.Optional; +import java.util.Set; import org.apache.nifi.serialization.record.RecordSchema; public class SchemaTextAsAttribute implements SchemaAccessWriter { + private static final Set schemaFields = EnumSet.of(SchemaField.SCHEMA_TEXT, SchemaField.SCHEMA_TEXT_FORMAT); @Override public void writeHeader(final RecordSchema schema, final OutputStream out) { @@ -49,4 +52,9 @@ public void validateSchema(final RecordSchema schema) throws SchemaNotFoundExcep throw new SchemaNotFoundException("Cannot write Schema Text as Attribute because the Schema's Text is not present"); } } + + @Override + public Set getRequiredSchemaFields() { + return schemaFields; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java index 5301b71d37a3..5ef18f84973f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java @@ -19,16 +19,22 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; import org.apache.nifi.controller.ConfigurationContext; import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceWriter; import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceWriter; import org.apache.nifi.schema.access.SchemaAccessWriter; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNameAsAttribute; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schema.access.SchemaTextAsAttribute; @@ -36,10 +42,10 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryService { - static final AllowableValue SCHEMA_NAME_ATTRIBUTE = new AllowableValue("schema-name", "Use 'schema.name' Attribute", + 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", "Use 'avro.schema' Attribute", + 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", @@ -68,7 +74,7 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic @Override protected List getSupportedPropertyDescriptors() { - final List properties = new ArrayList<>(super.getSupportedPropertyDescriptors()); + final List properties = new ArrayList<>(); final AllowableValue[] strategies = getSchemaWriteStrategyValues().toArray(new AllowableValue[0]); properties.add(new PropertyDescriptor.Builder() @@ -76,6 +82,7 @@ protected List getSupportedPropertyDescriptors() { .defaultValue(getDefaultSchemaWriteStrategy().getValue()) .allowableValues(strategies) .build()); + properties.addAll(super.getSupportedPropertyDescriptors()); return properties; } @@ -119,4 +126,34 @@ protected SchemaAccessWriter getSchemaWriteStrategy(final String allowableValue) return null; } + + protected Set getRequiredSchemaFields(final ValidationContext validationContext) { + final String writeStrategyValue = validationContext.getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + final SchemaAccessWriter writer = getSchemaWriteStrategy(writeStrategyValue); + final Set requiredFields = writer.getRequiredSchemaFields(); + return requiredFields; + } + + @Override + protected Collection customValidate(final ValidationContext validationContext) { + final List results = new ArrayList<>(super.customValidate(validationContext)); + + final Set suppliedFields = getSuppliedSchemaFields(validationContext); + final Set requiredFields = getRequiredSchemaFields(validationContext); + + final Set missingFields = new HashSet<>(requiredFields); + missingFields.removeAll(suppliedFields); + + if (!missingFields.isEmpty()) { + results.add(new ValidationResult.Builder() + .subject("Schema Access Strategy") + .valid(false) + .explanation("The configured Schema Write Strategy requires the " + missingFields.iterator().next() + + " but the configured Schema Access Strategy does not provide this information in conjunction with the selected Schema Registry. " + + "This Schema Access Strategy, as configured, cannot be used in conjunction with this Schema Write Strategy.") + .build()); + } + + return results; + } } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java index ed4d97dc2e4e..3aa336d7e250 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java @@ -24,6 +24,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Set; import org.apache.nifi.annotation.lifecycle.OnEnabled; import org.apache.nifi.avro.AvroSchemaValidator; @@ -39,6 +40,7 @@ import org.apache.nifi.schema.access.HortonworksAttributeSchemaReferenceStrategy; import org.apache.nifi.schema.access.HortonworksEncodedSchemaReferenceStrategy; import org.apache.nifi.schema.access.SchemaAccessStrategy; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNamePropertyStrategy; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.schemaregistry.services.SchemaRegistry; @@ -78,7 +80,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(true) .defaultValue("${schema.name}") - .required(true) + .required(false) .build(); static final PropertyDescriptor SCHEMA_TEXT = new PropertyDescriptor.Builder() @@ -88,7 +90,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService { .addValidator(new AvroSchemaValidator()) .expressionLanguageSupported(true) .defaultValue("${avro.schema}") - .required(true) + .required(false) .build(); @@ -181,6 +183,15 @@ protected List getSchemaAccessStrategyValues() { return strategyList; } + protected Set getSuppliedSchemaFields(final ValidationContext validationContext) { + final String accessStrategyValue = validationContext.getProperty(SCHEMA_ACCESS_STRATEGY).getValue(); + final SchemaRegistry schemaRegistry = validationContext.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class); + final SchemaAccessStrategy accessStrategy = getSchemaAccessStrategy(accessStrategyValue, schemaRegistry, validationContext); + + final Set suppliedFields = accessStrategy.getSuppliedSchemaFields(); + return suppliedFields; + } + protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry) { if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue())) { return new SchemaNamePropertyStrategy(schemaRegistry, getConfigurationContext().getProperty(SCHEMA_NAME)); @@ -195,4 +206,18 @@ protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableVal return null; } + protected SchemaAccessStrategy getSchemaAccessStrategy(final String allowableValue, final SchemaRegistry schemaRegistry, final ValidationContext context) { + if (allowableValue.equalsIgnoreCase(SCHEMA_NAME_PROPERTY.getValue())) { + return new SchemaNamePropertyStrategy(schemaRegistry, context.getProperty(SCHEMA_NAME)); + } else if (allowableValue.equalsIgnoreCase(SCHEMA_TEXT_PROPERTY.getValue())) { + return new AvroSchemaTextStrategy(context.getProperty(SCHEMA_TEXT)); + } else if (allowableValue.equalsIgnoreCase(HWX_CONTENT_ENCODED_SCHEMA.getValue())) { + return new HortonworksEncodedSchemaReferenceStrategy(schemaRegistry); + } else if (allowableValue.equalsIgnoreCase(HWX_SCHEMA_REF_ATTRIBUTES.getValue())) { + return new HortonworksAttributeSchemaReferenceStrategy(schemaRegistry); + } + + return null; + } + } diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java index 5c20f58450f7..88362b874628 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-schema-registry-service-api/src/main/java/org/apache/nifi/schemaregistry/services/SchemaRegistry.java @@ -17,10 +17,13 @@ package org.apache.nifi.schemaregistry.services; import java.io.IOException; +import java.util.Set; import org.apache.nifi.controller.ControllerService; +import org.apache.nifi.schema.access.SchemaField; import org.apache.nifi.schema.access.SchemaNotFoundException; import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; /** * Represents {@link ControllerService} strategy to expose internal and/or @@ -53,7 +56,10 @@ public interface SchemaRegistry extends ControllerService { String retrieveSchemaText(long schemaId, int version) throws IOException, SchemaNotFoundException; /** - * Retrieves and returns the RecordSchema based on the provided name of the schema available in Schema Registry. + * Retrieves and returns the RecordSchema based on the provided name of the schema available in Schema Registry. The RecordSchema + * that is returned must have the Schema's name populated in its SchemaIdentifier. I.e., a call to + * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getName() getName()} + * will always return an {@link Optional} that is not empty. * * @return the latest version of the schema with the given name, or null if no schema can be found with the given name. * @throws SchemaNotFoundException if unable to find the schema with the given name @@ -62,7 +68,11 @@ public interface SchemaRegistry extends ControllerService { /** - * Retrieves the schema with the given ID and version + * Retrieves the schema with the given ID and version. The RecordSchema that is returned must have the Schema's identifier and version + * populated in its SchemaIdentifier. I.e., a call to + * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getIdentifier() getIdentifier()} + * will always return an {@link Optional} that is not empty, as will a call to + * {@link RecordSchema}.{@link RecordSchema#getIdentifier() getIdentifier()}.{@link SchemaIdentifier#getVersion() getVersion()}. * * @param schemaId the unique identifier for the desired schema * @param version the version of the desired schema @@ -74,4 +84,8 @@ public interface SchemaRegistry extends ControllerService { */ RecordSchema retrieveSchema(long schemaId, int version) throws IOException, SchemaNotFoundException; + /** + * @return the set of all Schema Fields that are supplied by the RecordSchema that is returned from {@link #retrieveSchema(String)} and {@link #retrieveSchema(long, int)} + */ + Set getSuppliedSchemaFields(); } From 0d1bec208fa381ae4e26e7415ff8be728b134834 Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 24 Apr 2017 14:51:54 -0400 Subject: [PATCH 7/8] NIFI-3682: Implemented PR feedback --- .../apache/nifi/serialization/SimpleRecordSchema.java | 2 +- .../src/main/java/org/apache/nifi/grok/GrokReader.java | 9 +++++---- .../main/java/org/apache/nifi/grok/GrokRecordReader.java | 6 +++++- .../serialization/SchemaRegistryRecordSetWriter.java | 3 ++- .../apache/nifi/serialization/SchemaRegistryService.java | 7 ++++--- 5 files changed, 17 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java index f367e1b72223..017aef1e3803 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-service-api/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java @@ -51,7 +51,7 @@ public SimpleRecordSchema(final List fields, final String text, fin this(fields, text, schemaFormat, true, id); } - public SimpleRecordSchema(final List fields, final String text, final String schemaFormat, final boolean textAvailable, final SchemaIdentifier id) { + private SimpleRecordSchema(final List fields, final String text, final String schemaFormat, final boolean textAvailable, final SchemaIdentifier id) { this.text = text; this.schemaFormat = schemaFormat; this.schemaIdentifier = id; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java index 596f1dd07d7c..778c738bd87f 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokReader.java @@ -59,10 +59,9 @@ + "so that it can be processed. The service is configured using Grok patterns. " + "The service reads from a stream of data and splits each message that it finds into a separate Record, each containing the fields that are configured. " + "If a line in the input does not match the expected message pattern, the line of text is either considered to be part of the previous " - + "message or is skipped, depending on the configuration,, with the exception of stack traces. A stack trace that is found at the end of " + + "message or is skipped, depending on the configuration, with the exception of stack traces. A stack trace that is found at the end of " + "a log message is considered to be part of the previous message but is added to the 'stackTrace' field of the Record. If a record has " - + "no stack trace, it will have a NULL value for the stackTrace field. All fields that are parsed are considered to be of type String by default. " - + "If there is need to change the type of a field, this can be accomplished by configuring the Schema Registry to use and adding the appropriate schema.") + + "no stack trace, it will have a NULL value for the stackTrace field (assuming that the schema does in fact include a stackTrace field of type String).") public class GrokReader extends SchemaRegistryService implements RecordReaderFactory { private volatile Grok grok; private volatile boolean appendUnmatchedLine; @@ -76,7 +75,8 @@ public class GrokReader extends SchemaRegistryService implements RecordReaderFac "The line of text that does not match the Grok Expression will be skipped."); static final AllowableValue STRING_FIELDS_FROM_GROK_EXPRESSION = new AllowableValue("string-fields-from-grok-expression", "Use String Fields From Grok Expression", - "The schema will be derived by using the field names present in the Grok Expression. All fields will be assumed to be of type String."); + "The schema will be derived by using the field names present in the Grok Expression. All fields will be assumed to be of type String. Additionally, a field will be included " + + "with a name of 'stackTrace' and a type of String."); static final PropertyDescriptor PATTERN_FILE = new PropertyDescriptor.Builder() .name("Grok Pattern File") @@ -198,6 +198,7 @@ public Set getSuppliedSchemaFields() { } } + @Override public RecordReader createRecordReader(final FlowFile flowFile, final InputStream in, final ComponentLog logger) throws IOException, SchemaNotFoundException { final RecordSchema schema = getSchema(flowFile, in); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java index 6d3a16d8a3a0..5859f6f083af 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/grok/GrokRecordReader.java @@ -135,7 +135,11 @@ public Record nextRecord() throws IOException, MalformedRecordException { } if (append && toAppend.length() > 0) { - final String lastFieldBeforeStackTrace = schema.getFieldNames().get(schema.getFieldCount() - 2); + final String lastFieldName = schema.getField(schema.getFieldCount() - 1).getFieldName(); + + final int fieldIndex = STACK_TRACE_COLUMN_NAME.equals(lastFieldName) ? schema.getFieldCount() - 2 : schema.getFieldCount() - 1; + final String lastFieldBeforeStackTrace = schema.getFieldNames().get(fieldIndex); + final Object existingValue = values.get(lastFieldBeforeStackTrace); final String updatedValue = existingValue == null ? toAppend.toString() : existingValue + toAppend.toString(); values.put(lastFieldBeforeStackTrace, updatedValue); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java index 5ef18f84973f..3d924f76bc00 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java @@ -50,7 +50,8 @@ public abstract class SchemaRegistryRecordSetWriter extends SchemaRegistryServic + "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", "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. " + + "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", diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java index 3aa336d7e250..2cee613d9721 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java @@ -55,7 +55,8 @@ public abstract class SchemaRegistryService extends AbstractControllerService { + "If Expression Language is used, the value of the 'Schema Text' property must be valid after substituting the expressions."); static final AllowableValue HWX_CONTENT_ENCODED_SCHEMA = new AllowableValue("hwx-content-encoded-schema", "HWX Content-Encoded Schema Reference", "The content of the FlowFile contains 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."); + + "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, " + + "found at https://github.com/hortonworks/registry"); static final AllowableValue HWX_SCHEMA_REF_ATTRIBUTES = new AllowableValue("hwx-schema-ref-attributes", "HWX Schema Reference Attributes", "The FlowFile contains 3 Attributes that will be used to lookup a Schema from the configured Schema Registry: 'schema.identifier', 'schema.version', and 'schema.protocol.version'"); @@ -70,7 +71,7 @@ public abstract class SchemaRegistryService extends AbstractControllerService { .name("Schema Access Strategy") .description("Specifies how to obtain the schema that is to be used for interpreting the data.") .allowableValues(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA) - .defaultValue(SCHEMA_TEXT_PROPERTY.getValue()) + .defaultValue(SCHEMA_NAME_PROPERTY.getValue()) .required(true) .build(); @@ -119,7 +120,7 @@ protected List getSupportedPropertyDescriptors() { } protected AllowableValue getDefaultSchemaAccessStrategy() { - return SCHEMA_TEXT_PROPERTY; + return SCHEMA_NAME_PROPERTY; } @OnEnabled From d4d5856c9327b79b85f1c3d8013918ebc896303d Mon Sep 17 00:00:00 2001 From: Mark Payne Date: Mon, 24 Apr 2017 15:59:17 -0400 Subject: [PATCH 8/8] NIFI-3682: Fixed validation logic for GrokReader when defining schema based on Grok Expression --- .../serialization/SchemaRegistryRecordSetWriter.java | 8 ++++++-- .../nifi/serialization/SchemaRegistryService.java | 11 +++++++---- 2 files changed, 13 insertions(+), 6 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java index 3d924f76bc00..c9daded18688 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryRecordSetWriter.java @@ -92,11 +92,15 @@ protected AllowableValue getDefaultSchemaWriteStrategy() { return AVRO_SCHEMA_ATTRIBUTE; } + protected PropertyDescriptor getSchemaWriteStrategyDescriptor() { + return getPropertyDescriptor(SCHEMA_WRITE_STRATEGY.getName()); + } + @OnEnabled public void storeSchemaWriteStrategy(final ConfigurationContext context) { this.configurationContext = context; - final String writerValue = context.getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + final String writerValue = context.getProperty(getSchemaWriteStrategyDescriptor()).getValue(); this.schemaAccessWriter = getSchemaWriteStrategy(writerValue); } @@ -129,7 +133,7 @@ protected SchemaAccessWriter getSchemaWriteStrategy(final String allowableValue) } protected Set getRequiredSchemaFields(final ValidationContext validationContext) { - final String writeStrategyValue = validationContext.getProperty(SCHEMA_WRITE_STRATEGY).getValue(); + final String writeStrategyValue = validationContext.getProperty(getSchemaWriteStrategyDescriptor()).getValue(); final SchemaAccessWriter writer = getSchemaWriteStrategy(writeStrategyValue); final Set requiredFields = writer.getRequiredSchemaFields(); return requiredFields; diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java index 2cee613d9721..0988935e9059 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/serialization/SchemaRegistryService.java @@ -100,6 +100,9 @@ public abstract class SchemaRegistryService extends AbstractControllerService { private final List strategyList = Collections.unmodifiableList(Arrays.asList(SCHEMA_NAME_PROPERTY, SCHEMA_TEXT_PROPERTY, HWX_SCHEMA_REF_ATTRIBUTES, HWX_CONTENT_ENCODED_SCHEMA)); + private PropertyDescriptor getSchemaAcessStrategyDescriptor() { + return getPropertyDescriptor(SCHEMA_ACCESS_STRATEGY.getName()); + } @Override protected List getSupportedPropertyDescriptors() { @@ -129,7 +132,7 @@ public void storeSchemaAccessStrategy(final ConfigurationContext context) { final SchemaRegistry schemaRegistry = context.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class); - final PropertyDescriptor descriptor = getPropertyDescriptor(SCHEMA_ACCESS_STRATEGY.getName()); + final PropertyDescriptor descriptor = getSchemaAcessStrategyDescriptor(); final String schemaAccess = context.getProperty(descriptor).getValue(); this.schemaAccessStrategy = getSchemaAccessStrategy(schemaAccess, schemaRegistry); } @@ -147,7 +150,7 @@ public RecordSchema getSchema(final FlowFile flowFile, final InputStream content } private String getSchemaAccessStrategyName(final String schemaAccessValue) { - for (final AllowableValue allowableValue : SCHEMA_ACCESS_STRATEGY.getAllowableValues()) { + for (final AllowableValue allowableValue : getSchemaAcessStrategyDescriptor().getAllowableValues()) { if (allowableValue.getValue().equalsIgnoreCase(schemaAccessValue)) { return allowableValue.getDisplayName(); } @@ -163,7 +166,7 @@ private boolean isSchemaRegistryRequired(final String schemaAccessValue) { @Override protected Collection customValidate(final ValidationContext validationContext) { - final String schemaAccessStrategy = validationContext.getProperty(SCHEMA_ACCESS_STRATEGY).getValue(); + final String schemaAccessStrategy = validationContext.getProperty(getSchemaAcessStrategyDescriptor()).getValue(); if (isSchemaRegistryRequired(schemaAccessStrategy)) { final boolean registrySet = validationContext.getProperty(SCHEMA_REGISTRY).isSet(); if (!registrySet) { @@ -185,7 +188,7 @@ protected List getSchemaAccessStrategyValues() { } protected Set getSuppliedSchemaFields(final ValidationContext validationContext) { - final String accessStrategyValue = validationContext.getProperty(SCHEMA_ACCESS_STRATEGY).getValue(); + final String accessStrategyValue = validationContext.getProperty(getSchemaAcessStrategyDescriptor()).getValue(); final SchemaRegistry schemaRegistry = validationContext.getProperty(SCHEMA_REGISTRY).asControllerService(SchemaRegistry.class); final SchemaAccessStrategy accessStrategy = getSchemaAccessStrategy(accessStrategyValue, schemaRegistry, validationContext);