nifi git commit: NIFI-5781 - Incorrect schema for provenance events in SiteToSiteProvenanceReportingTask
Repository: nifi Updated Branches: refs/heads/master f0a7dda84 -> bcfd6f0b1 NIFI-5781 - Incorrect schema for provenance events in SiteToSiteProvenanceReportingTask Signed-off-by: Matthew Burgess This closes #3123 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/bcfd6f0b Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/bcfd6f0b Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/bcfd6f0b Branch: refs/heads/master Commit: bcfd6f0b13ec9bf595652c8cf11be55b63b30ba7 Parents: f0a7dda Author: Pierre Villard Authored: Thu Nov 1 15:19:06 2018 +0100 Committer: Matthew Burgess Committed: Thu Nov 1 10:58:46 2018 -0400 -- .../additionalDetails.html | 32 ++-- .../src/main/resources/schema-provenance.avsc | 32 ++-- 2 files changed, 32 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/bcfd6f0b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask/additionalDetails.html -- diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask/additionalDetails.html b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask/additionalDetails.html index 676674e..fee9e93 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask/additionalDetails.html +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/docs/org.apache.nifi.reporting.SiteToSiteProvenanceReportingTask/additionalDetails.html @@ -40,9 +40,9 @@ { - "namespace": "nifi", - "name": "provenanceEvent", - "type": "record", + "type" : "record", + "name" : "provenance", + "namespace" : "provenance", "fields": [ { "name": "eventId", "type": "string" }, { "name": "eventOrdinal", "type": "long" }, @@ -50,27 +50,27 @@ { "name": "timestampMillis", "type": "long" }, { "name": "durationMillis", "type": "long" }, { "name": "lineageStart", "type": { "type": "long", "logicalType": "timestamp-millis" } }, -{ "name": "details", "type": "string" }, -{ "name": "componentId", "type": "string" }, -{ "name": "componentType", "type": "string" }, -{ "name": "componentName", "type": "string" }, -{ "name": "processGroupId", "type": "string" }, -{ "name": "processGroupName", "type": "string" }, -{ "name": "entityId", "type": "string" }, -{ "name": "entityType", "type": "string" }, +{ "name": "details", "type": ["null", "string"] }, +{ "name": "componentId", "type": ["null", "string"] }, +{ "name": "componentType", "type": ["null", "string"] }, +{ "name": "componentName", "type": ["null", "string"] }, +{ "name": "processGroupId", "type": ["null", "string"] }, +{ "name": "processGroupName", "type": ["null", "string"] }, +{ "name": "entityId", "type": ["null", "string"] }, +{ "name": "entityType", "type": ["null", "string"] }, { "name": "entitySize", "type": ["null", "long"] }, { "name": "previousEntitySize", "type": ["null", "long"] }, { "name": "updatedAttributes", "type": { "type": "map", "values": "string" } }, { "name": "previousAttributes", "type": { "type": "map", "values": "string" } }, -{ "name": "actorHostname", "type": "string" }, -{ "name": "contentURI", "type": "string" }, -{ "name": "previousContentURI", "type": "string" }, +{ "name": "actorHostname", "type": ["null", "string"] }, +{ "name": "contentURI", "type": ["null", "string"] }, +{ "name": "previousContentURI", "type": ["null", "string"] }, { "name": "parentIds", "type": { "type": "array", "items": "string" } }, { "name": "childIds", "type": { "type": "array", "items": "string" } }, { "name": "platform", "type": "string" }, { "name": "application", "type": "string" }, -{ "name": "remoteIdentifier", "type": "string" }, -{ "name": "alternateIdentifier", "type": "string" }, +{ "name": "remoteIdentifier", "type": ["null", "string"] }, +{ "name": "alternateIdentifier", "type": ["null", "string"] }, { "name": "transitUri", "type": ["null", "string"] } ] } http://git-wip-us.apache.org/repos/asf/nifi/blob/bcfd6f0b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/resources/schema-provenance.avsc
nifi git commit: NIFI-5788: Introduce batch size limit in PutDatabaseRecord processor
Repository: nifi Updated Branches: refs/heads/master 13011ac6d -> d319a3ef2 NIFI-5788: Introduce batch size limit in PutDatabaseRecord processor NIFI-5788: Introduce batch size limit in PutDatabaseRecord processor Renamed 'batch size' to 'Maximum Batch Size'. Changed default value of max_batch_size to zero (INFINITE) Fixed parameter validation. Added unit tests Signed-off-by: Matthew Burgess This closes #3128 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/d319a3ef Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/d319a3ef Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/d319a3ef Branch: refs/heads/master Commit: d319a3ef2f14317f29a1be5a189bc34f8b3fdbd6 Parents: 13011ac Author: vadimar Authored: Mon Nov 5 13:15:12 2018 +0200 Committer: Matthew Burgess Committed: Thu Nov 15 10:31:34 2018 -0500 -- .../processors/standard/PutDatabaseRecord.java | 29 +- .../standard/TestPutDatabaseRecord.groovy | 103 +++ 2 files changed, 130 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/d319a3ef/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java index 2f2d901..d79cf3c 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/PutDatabaseRecord.java @@ -275,6 +275,17 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { .required(true) .build(); +static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder() +.name("put-db-record-max-batch-size") +.displayName("Maximum Batch Size") +.description("Specifies maximum batch size for INSERT and UPDATE statements. This parameter has no effect for other statements specified in 'Statement Type'." ++ " Zero means the batch size is not limited.") +.defaultValue("0") +.required(false) +.addValidator(StandardValidators.NON_NEGATIVE_INTEGER_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES) +.build(); + protected static List propDescriptors; private Cache schemaCache; @@ -303,6 +314,7 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { pds.add(QUERY_TIMEOUT); pds.add(RollbackOnFailure.ROLLBACK_ON_FAILURE); pds.add(TABLE_SCHEMA_CACHE_SIZE); +pds.add(MAX_BATCH_SIZE); propDescriptors = Collections.unmodifiableList(pds); } @@ -641,6 +653,10 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { Record currentRecord; List fieldIndexes = sqlHolder.getFieldIndexes(); +final Integer maxBatchSize = context.getProperty(MAX_BATCH_SIZE).evaluateAttributeExpressions(flowFile).asInteger(); +int currentBatchSize = 0; +int batchIndex = 0; + while ((currentRecord = recordParser.nextRecord()) != null) { Object[] values = currentRecord.getValues(); if (values != null) { @@ -667,11 +683,20 @@ public class PutDatabaseRecord extends AbstractSessionFactoryProcessor { } } ps.addBatch(); +if (++currentBatchSize == maxBatchSize) { +batchIndex++; +log.debug("Executing query {}; fieldIndexes: {}; batch index: {}; batch size: {}", new Object[]{sqlHolder.getSql(), sqlHolder.getFieldIndexes(), batchIndex, currentBatchSize}); +ps.executeBatch(); +currentBatchSize = 0; +} } } -log.debug("Executing query {}", new Object[]{sqlHolder}); -ps.executeBatch(); +if (currentBatchSize > 0) { +batchIndex++; +log.debug("Executing query {}; fieldIndexes: {}; batch index: {}; batch size: {}", new Object[]{sqlHolder.getSql(), sqlHolder.getFieldIndexes(), batchIndex, currentBatchSize}); +
nifi git commit: NIFI-4130 Add lookup controller service in TransformXML to define XSLT from the UI
Repository: nifi Updated Branches: refs/heads/master 9e7610ac7 -> 4112af013 NIFI-4130 Add lookup controller service in TransformXML to define XSLT from the UI addressed review comments Signed-off-by: Matthew Burgess This closes #1953 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/4112af01 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/4112af01 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/4112af01 Branch: refs/heads/master Commit: 4112af013d1b1f49e83f881d85ebe66e097840b5 Parents: 9e7610a Author: Pierre Villard Authored: Mon Jun 26 17:48:06 2017 +0200 Committer: Matthew Burgess Committed: Wed Nov 14 09:45:20 2018 -0500 -- .../nifi/processors/standard/TransformXml.java | 132 +++--- .../processors/standard/TestTransformXml.java | 133 ++- 2 files changed, 238 insertions(+), 27 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/4112af01/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java index ff15428..1cc57fa 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/TransformXml.java @@ -16,21 +16,24 @@ */ package org.apache.nifi.processors.standard; -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; import java.io.File; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.charset.StandardCharsets; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; + import javax.xml.XMLConstants; import javax.xml.transform.OutputKeys; import javax.xml.transform.Templates; @@ -39,6 +42,8 @@ import javax.xml.transform.TransformerConfigurationException; import javax.xml.transform.TransformerFactory; import javax.xml.transform.stream.StreamResult; import javax.xml.transform.stream.StreamSource; + +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -49,6 +54,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyValue; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.components.Validator; @@ -56,6 +62,9 @@ import org.apache.nifi.expression.AttributeExpression; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.lookup.LookupFailureException; +import org.apache.nifi.lookup.LookupService; +import org.apache.nifi.lookup.StringLookupService; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; @@ -67,6 +76,10 @@ import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.util.StopWatch; import org.apache.nifi.util.Tuple; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + @EventDriven @SideEffectFree @SupportsBatching @@ -82,12 +95,33 @@ public class TransformXml extends AbstractProcessor { public static final PropertyDescriptor XSLT_FILE_NAME = new PropertyDescriptor.Builder() .name("XSLT file name") -.description("Provides the name (including full path) of the XSLT file to apply to the flowfile XML content.") -.required(true) +.description("Provides the name (including full path) of
[1/2] nifi git commit: NIFI-5533: Be more efficient with heap utilization - Updated FlowFile Repo / Write Ahead Log so that any update that writes more than 1 MB of data is written to a file inside th
Repository: nifi Updated Branches: refs/heads/master c87d79193 -> c425bd288 http://git-wip-us.apache.org/repos/asf/nifi/blob/c425bd28/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 3496795..4354dc4 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -68,6 +68,7 @@ import java.io.OutputStream; import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.HashMap; @@ -98,6 +99,11 @@ import java.util.stream.Collectors; * */ public final class StandardProcessSession implements ProcessSession, ProvenanceEventEnricher { +private static final int SOURCE_EVENT_BIT_INDEXES = (1 << ProvenanceEventType.CREATE.ordinal()) +| (1 << ProvenanceEventType.FORK.ordinal()) +| (1 << ProvenanceEventType.JOIN.ordinal()) +| (1 << ProvenanceEventType.RECEIVE.ordinal()) +| (1 << ProvenanceEventType.FETCH.ordinal()); private static final AtomicLong idGenerator = new AtomicLong(0L); private static final AtomicLong enqueuedIndex = new AtomicLong(0L); @@ -110,7 +116,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE private static final Logger claimLog = LoggerFactory.getLogger(StandardProcessSession.class.getSimpleName() + ".claims"); private static final int MAX_ROLLBACK_FLOWFILES_TO_LOG = 5; -private final Map records = new ConcurrentHashMap<>(); +private final Map records = new ConcurrentHashMap<>(); private final Map connectionCounts = new ConcurrentHashMap<>(); private final Map> unacknowledgedFlowFiles = new ConcurrentHashMap<>(); private final Map appendableStreams = new ConcurrentHashMap<>(); @@ -253,7 +259,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE List autoTerminatedEvents = null; // validate that all records have a transfer relationship for them and if so determine the destination node and clone as necessary -final Map toAdd = new HashMap<>(); +final Map toAdd = new HashMap<>(); for (final StandardRepositoryRecord record : records.values()) { if (record.isMarkedForDelete()) { continue; @@ -317,7 +323,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE newRecord.setDestination(destination.getFlowFileQueue()); newRecord.setTransferRelationship(record.getTransferRelationship()); // put the mapping into toAdd because adding to records now will cause a ConcurrentModificationException -toAdd.put(clone, newRecord); +toAdd.put(clone.getId(), newRecord); } } } @@ -365,10 +371,7 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE * points to the Original Claim -- which has already been removed! * */ -for (final Map.Entry entry : checkpoint.records.entrySet()) { -final FlowFile flowFile = entry.getKey(); -final StandardRepositoryRecord record = entry.getValue(); - +for (final StandardRepositoryRecord record : checkpoint.records.values()) { if (record.isMarkedForDelete()) { // if the working claim is not the same as the original claim, we can immediately destroy the working claim // because it was created in this session and is to be deleted. We don't need to wait for the FlowFile Repo to sync. @@ -380,10 +383,14 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE // an issue if we only updated the FlowFile attributes. decrementClaimCount(record.getOriginalClaim()); } -final long flowFileLife = System.currentTimeMillis() - flowFile.getEntryDate(); -final Connectable connectable = context.getConnectable(); -final Object terminator = connectable
[2/2] nifi git commit: NIFI-5533: Be more efficient with heap utilization - Updated FlowFile Repo / Write Ahead Log so that any update that writes more than 1 MB of data is written to a file inside th
NIFI-5533: Be more efficient with heap utilization - Updated FlowFile Repo / Write Ahead Log so that any update that writes more than 1 MB of data is written to a file inside the FlowFile Repo rather than being buffered in memory - Update SplitText so that it does not hold FlowFiles that are not the latest version in heap. Doing them from being garbage collected, so while the Process Session is holding the latest version of the FlowFile, SplitText is holding an older version, and this results in two copies of the same FlowFile object NIFI-5533: Checkpoint NIFI-5533: Bug Fixes Signed-off-by: Matthew Burgess This closes #2974 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c425bd28 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c425bd28 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c425bd28 Branch: refs/heads/master Commit: c425bd2880dc2c45c96e0dfcc4990f1e20e14d0a Parents: c87d791 Author: Mark Payne Authored: Fri Aug 17 14:08:14 2018 -0400 Committer: Matthew Burgess Committed: Tue Oct 9 09:18:02 2018 -0400 -- .../language/StandardPreparedQuery.java | 21 +- .../nifi/repository/schema/RecordIterator.java | 28 +++ .../repository/schema/SchemaRecordReader.java | 68 +- .../repository/schema/SchemaRecordWriter.java | 9 +- .../repository/schema/SingleRecordIterator.java | 45 .../nifi/wali/LengthDelimitedJournal.java | 119 ++- .../wali/SequentialAccessWriteAheadLog.java | 6 +- .../org/apache/nifi/wali/WriteAheadJournal.java | 5 + .../org/wali/MinimalLockingWriteAheadLog.java | 3 + .../src/main/java/org/wali/SerDe.java | 34 +++ .../java/org/wali/WriteAheadRepository.java | 2 +- .../wali/TestSequentialAccessWriteAheadLog.java | 89 ++-- .../test/java/org/wali/DummyRecordSerde.java| 83 .../WriteAvroSchemaAttributeStrategy.java | 22 +- .../repository/SchemaRepositoryRecordSerde.java | 71 ++- .../repository/FileSystemRepository.java| 2 +- .../repository/StandardProcessSession.java | 210 ++- .../nifi/processor/StandardProcessContext.java | 36 +++- .../repository/StandardRepositoryRecord.java| 45 +++- .../nifi/processors/standard/SplitText.java | 87 .../processors/attributes/UpdateAttribute.java | 69 -- 21 files changed, 828 insertions(+), 226 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/c425bd28/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/StandardPreparedQuery.java -- 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 cf90d8d..fc3f9b7 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 @@ -16,12 +16,6 @@ */ package org.apache.nifi.attribute.expression.language; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - import org.apache.nifi.attribute.expression.language.evaluation.Evaluator; import org.apache.nifi.attribute.expression.language.evaluation.literals.StringLiteralEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.selection.AllAttributesEvaluator; @@ -34,7 +28,14 @@ import org.apache.nifi.attribute.expression.language.evaluation.selection.MultiN import org.apache.nifi.expression.AttributeValueDecorator; import org.apache.nifi.processor.exception.ProcessException; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + public class StandardPreparedQuery implements PreparedQuery { +private static final String EMPTY_STRING = ""; private final List expressions; private volatile VariableImpact variableImpact; @@ -45,6 +46,14 @@ public class StandardPreparedQuery implements PreparedQuery { @Override public String evaluateExpressions(final Map valMap, final AttributeValueDecorator decorator, final Map stateVariables) throws ProcessException { +if (expressions.isEmpty()) { +return EMPTY_STRING; +} +if (expressions.size() == 1) { +final String evaluated = expressions.get(0).evaluate(valMap, decorator, stateVariables); +return evaluated == null ? EMPTY_STRING
nifi git commit: NIFI-5051 Created ElasticSearch lookup service.
Repository: nifi Updated Branches: refs/heads/master 748cf7456 -> b1478cdb1 NIFI-5051 Created ElasticSearch lookup service. NIFI-5051 Fixed checkstyle issue. NIFI-5051 Converted ES lookup service to use a SchemaRegistry. NIFI-5051 Cleaned up POM and added a simple unit test that uses a mock client service. NIFI-5051 Added change; waiting for feedback. NIFI-5051 Changed query setup based on code review. Changed tests to Groovy to make them easier to read with all of the inline JSON. NIFI-5051 fixed a checkstyle issue. NIFI-5051 Rebased to cleanup merge issues NIFI-5051 Added changes from a code review. NIFI-5051 Fixed a checkstyle issue. NIFI-5051 Added coverage generator for tests. Rebased. NIFI-5051 Updated service and switched it over to JsonInferenceSchemaRegistryService. NIFI-5051 Removed dead code. NIFI-5051 Fixed checkstyle errors. NIFI-5051 Refactored query builder. NIFI-5051 Added placeholder gitignore to force test compile. NIFI-5051 Added note explaining why the .gitignore file was needed. NIFI-5051 Made constructor public. NIFI-5051 Fixed path issue in client service integration tests. NIFI-5051 Added additional mapping capabilities to let users massage the result set into the fields they want. Signed-off-by: Matthew Burgess This closes #2615 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/b1478cdb Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/b1478cdb Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/b1478cdb Branch: refs/heads/master Commit: b1478cdb195722d7d285cad53d1f89c325868f55 Parents: 748cf74 Author: Mike Thomsen Authored: Fri Apr 6 21:38:07 2018 -0400 Committer: Matthew Burgess Committed: Mon Oct 1 10:48:45 2018 -0400 -- .../nifi-elasticsearch-client-service/pom.xml | 35 +++ .../ElasticSearchLookupService.java | 315 +++ ...org.apache.nifi.controller.ControllerService | 1 + .../additionalDetails.html | 53 .../ElasticSearch5ClientService_IT.groovy | 148 + .../ElasticSearchLookupServiceTest.groovy | 69 .../ElasticSearchLookupService_IT.groovy| 211 + .../TestControllerServiceProcessor.groovy | 51 +++ .../TestElasticSearchClientService.groovy | 75 + .../integration/TestSchemaRegistry.groovy | 41 +++ .../src/test/java/.gitignore| 1 + .../ElasticSearchClientService_IT.java | 165 -- .../TestControllerServiceProcessor.java | 49 --- .../src/test/resources/setup.script | 11 +- 14 files changed, 1010 insertions(+), 215 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/b1478cdb/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml -- diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml index 8eb976d..b0c7842 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml @@ -58,6 +58,12 @@ +org.apache.nifi +nifi-lookup-service-api +provided + + + com.fasterxml.jackson.core jackson-databind 2.9.5 @@ -123,6 +129,33 @@ + +org.apache.nifi +nifi-avro-record-utils +1.8.0-SNAPSHOT +compile + + +org.apache.nifi +nifi-schema-registry-service-api +compile + + +com.jayway.jsonpath +json-path +2.4.0 + + +org.mockito +mockito-all +test + + +org.apache.nifi +nifi-record-path +1.8.0-SNAPSHOT +compile + @@ -140,6 +173,8 @@ 9400 5.6.2 90 +ERROR + ${project.basedir}/src/test/resources/setup.script http://git-wip-us.apache.org/repos/asf/nifi/blob/b1478cdb/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchLookupService.java -- diff --git
nifi git commit: NIFI-5609: Fixed NullPointer when attempting to view status history for a component that has not yet run
Repository: nifi Updated Branches: refs/heads/master f570cb980 -> 3dd548e80 NIFI-5609: Fixed NullPointer when attempting to view status history for a component that has not yet run Signed-off-by: Matthew Burgess This closes #3012 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/3dd548e8 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/3dd548e8 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/3dd548e8 Branch: refs/heads/master Commit: 3dd548e807c80749a99f83e18857cab879cd9c21 Parents: f570cb9 Author: Mark Payne Authored: Wed Sep 19 16:17:29 2018 -0400 Committer: Matthew Burgess Committed: Wed Sep 19 16:56:42 2018 -0400 -- .../VolatileComponentStatusRepository.java | 24 +++- 1 file changed, 23 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/3dd548e8/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java -- diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java index 5336d17..b280214 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/history/VolatileComponentStatusRepository.java @@ -27,6 +27,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.util.Arrays; +import java.util.Collections; import java.util.Date; import java.util.HashMap; import java.util.List; @@ -166,13 +167,34 @@ public class VolatileComponentStatusRepository implements ComponentStatusReposit private synchronized StatusHistory getStatusHistory(final String componentId, final boolean includeCounters, final Set> defaultMetricDescriptors) { final ComponentStatusHistory history = componentStatusHistories.get(componentId); if (history == null) { -return null; +return createEmptyStatusHistory(); } final List dates = timestamps.asList(); return history.toStatusHistory(dates, includeCounters, defaultMetricDescriptors); } +private StatusHistory createEmptyStatusHistory() { +final Date dateGenerated = new Date(); + +return new StatusHistory() { +@Override +public Date getDateGenerated() { +return dateGenerated; +} + +@Override +public Map getComponentDetails() { +return Collections.emptyMap(); +} + +@Override +public List getStatusSnapshots() { +return Collections.emptyList(); +} +}; +} + @Override public GarbageCollectionHistory getGarbageCollectionHistory(final Date start, final Date end) {
nifi git commit: NIFI-5598: Allow JMS Processors to lookup Connection Factory via JNDI
Repository: nifi Updated Branches: refs/heads/master 0da4f50ea -> ad80f5f06 NIFI-5598: Allow JMS Processors to lookup Connection Factory via JNDI NIFI-5598: Expose JNDI Principal & Credentails as explicit properties Signed-off-by: Matthew Burgess This closes #3005 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/ad80f5f0 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/ad80f5f0 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/ad80f5f0 Branch: refs/heads/master Commit: ad80f5f064a2895390bdff77278162a1fba68543 Parents: 0da4f50 Author: Mark Payne Authored: Fri Sep 14 10:48:59 2018 -0400 Committer: Matthew Burgess Committed: Wed Sep 19 12:56:58 2018 -0400 -- .../cf/JndiJmsConnectionFactoryProvider.java| 194 +++ ...org.apache.nifi.controller.ControllerService | 2 + .../additionalDetails.html | 94 + 3 files changed, 290 insertions(+) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/ad80f5f0/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java -- diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java new file mode 100644 index 000..876d933 --- /dev/null +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java @@ -0,0 +1,194 @@ +/* + * 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.jms.cf; + +import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.documentation.CapabilityDescription; +import org.apache.nifi.annotation.documentation.SeeAlso; +import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnDisabled; +import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.PropertyDescriptor.Builder; +import org.apache.nifi.components.Validator; +import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; + +import javax.jms.ConnectionFactory; +import javax.naming.Context; +import javax.naming.InitialContext; +import javax.naming.NamingException; +import java.util.Arrays; +import java.util.Hashtable; +import java.util.List; + +import static org.apache.nifi.processor.util.StandardValidators.NON_EMPTY_VALIDATOR; + +@Tags({"jms", "jndi", "messaging", "integration", "queue", "topic", "publish", "subscribe"}) +@CapabilityDescription("Provides a service to lookup an existing JMS ConnectionFactory using the Java Naming and Directory Interface (JNDI).") +@DynamicProperty( +description = "In order to perform a JNDI Lookup, an Initial Context must be established. When this is done, an Environment can be established for the context. Any dynamic/user-defined property" + +" that is added to this Controller Service will be added as an Environment configuration/variable to this Context.", +name = "The name of a JNDI Initial Context environment variable.", +value = "The value of the JNDI Initial Context Environment variable.", +expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY) +@SeeAlso(classNames = {"org.apache.nifi.jms.processors.ConsumeJMS", "org.apache.nifi.jms.processors.PublishJMS", "org.apache.nifi.jms.cf.JMSConnectionFactoryProvider"}) +public class JndiJmsConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition{ + +static final PropertyDescriptor INITIAL_NAMING_FACTORY_CLASS = new
nifi git commit: NIFI-5640: Improved efficiency of Avro Reader and some methods of AvroTypeUtil. Also switched ServiceStateTransition to using read/write locks instead of synchronized blocks because p
Repository: nifi Updated Branches: refs/heads/master ad4c886fb -> 2e1005e88 NIFI-5640: Improved efficiency of Avro Reader and some methods of AvroTypeUtil. Also switched ServiceStateTransition to using read/write locks instead of synchronized blocks because profiling showed that significant time was spent in determining state of a Controller Service when attempting to use it. Switching to a ReadLock should provide better performance there. Signed-off-by: Matthew Burgess This closes #3036 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/2e1005e8 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/2e1005e8 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/2e1005e8 Branch: refs/heads/master Commit: 2e1005e884cef70ea9c2eb1152d70e546ad2b5c3 Parents: ad4c886 Author: Mark Payne Authored: Thu Sep 27 10:10:48 2018 -0400 Committer: Matthew Burgess Committed: Thu Sep 27 15:38:47 2018 -0400 -- .../nifi/serialization/SimpleRecordSchema.java | 39 -- .../java/org/apache/nifi/avro/AvroTypeUtil.java | 126 ++- .../service/ServiceStateTransition.java | 85 + .../java/org/apache/nifi/avro/AvroReader.java | 19 ++- .../nifi/avro/AvroReaderWithEmbeddedSchema.java | 12 +- .../nifi/avro/AvroReaderWithExplicitSchema.java | 17 ++- .../apache/nifi/avro/NonCachingDatumReader.java | 65 ++ 7 files changed, 241 insertions(+), 122 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/2e1005e8/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java -- diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java index 5b85f03..6926c93 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/SimpleRecordSchema.java @@ -17,24 +17,25 @@ package org.apache.nifi.serialization; +import org.apache.nifi.serialization.record.DataType; +import org.apache.nifi.serialization.record.RecordField; +import org.apache.nifi.serialization.record.RecordSchema; +import org.apache.nifi.serialization.record.SchemaIdentifier; + import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Optional; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; -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 List fields = null; private Map fieldMap = null; private final boolean textAvailable; -private final String text; +private final AtomicReference text = new AtomicReference<>(); private final String schemaFormat; private final SchemaIdentifier schemaIdentifier; @@ -50,6 +51,10 @@ public class SimpleRecordSchema implements RecordSchema { this(text, schemaFormat, true, id); } +public SimpleRecordSchema(final SchemaIdentifier id) { +this(null, null, false, id); +} + public SimpleRecordSchema(final List fields, final String text, final String schemaFormat, final SchemaIdentifier id) { this(fields, text, schemaFormat, true, id); } @@ -60,7 +65,7 @@ public class SimpleRecordSchema implements RecordSchema { } private SimpleRecordSchema(final String text, final String schemaFormat, final boolean textAvailable, final SchemaIdentifier id) { -this.text = text; +this.text.set(text); this.schemaFormat = schemaFormat; this.schemaIdentifier = id; this.textAvailable = textAvailable; @@ -69,7 +74,7 @@ public class SimpleRecordSchema implements RecordSchema { @Override public Optional getSchemaText() { if (textAvailable) { -return Optional.ofNullable(text); +return Optional.ofNullable(text.get()); } else { return Optional.empty(); } @@ -121,13 +126,13 @@ public class SimpleRecordSchema implements RecordSchema { @Override public List getDataTypes() { -return getFields().stream().map(recordField -> recordField.getDataType()) +return getFields().stream().map(RecordField::getDataType) .collect(Collectors.toList()); } @Override public List getFieldNames() { -return
nifi git commit: NIFI-5612: Support JDBC drivers that return Long for unsigned ints
Repository: nifi Updated Branches: refs/heads/master e24388aa7 -> 0dd382370 NIFI-5612: Support JDBC drivers that return Long for unsigned ints Refactors tests in order to share code repeated in tests and to enable some parameterized testing. MySQL Connector/J 5.1.x in conjunction with MySQL 5.0.x will return a Long for ResultSet#getObject when the SQL type is an unsigned integer. This change prevents that error from occurring while implementing a more informational exception describing what the failing object's POJO type is in addition to its string value. Signed-off-by: Matthew Burgess This closes #3032 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/0dd38237 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/0dd38237 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/0dd38237 Branch: refs/heads/master Commit: 0dd382370bf139e0f8c1b22761e4aa306943dd77 Parents: e24388a Author: Colin Dean Authored: Wed Sep 19 20:27:47 2018 -0400 Committer: Matthew Burgess Committed: Fri Sep 28 13:46:04 2018 -0400 -- .../processors/standard/util/JdbcCommon.java| 25 ++- .../standard/util/JdbcCommonTestUtils.java | 60 .../standard/util/TestJdbcCommon.java | 84 +++--- .../util/TestJdbcCommonConvertToAvro.java | 152 +++ 4 files changed, 293 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/0dd38237/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java index 03761c6..9681e2f 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/util/JdbcCommon.java @@ -91,11 +91,13 @@ import org.apache.avro.SchemaBuilder.FieldAssembler; import org.apache.avro.SchemaBuilder.NullDefault; import org.apache.avro.SchemaBuilder.UnionAccumulator; import org.apache.avro.file.CodecFactory; +import org.apache.avro.UnresolvedUnionException; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; +import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.avro.AvroTypeUtil; import org.apache.nifi.components.PropertyDescriptor; @@ -449,8 +451,11 @@ public class JdbcCommon { } else { rec.put(i - 1, value); } +} else if ((value instanceof Long) && meta.getPrecision(i) < MAX_DIGITS_IN_INT) { +int intValue = ((Long)value).intValue(); +rec.put(i-1, intValue); } else { -rec.put(i - 1, value); +rec.put(i-1, value); } } else if (value instanceof Date) { @@ -470,8 +475,22 @@ public class JdbcCommon { rec.put(i - 1, value.toString()); } } -dataFileWriter.append(rec); -nrOfRows += 1; +try { +dataFileWriter.append(rec); +nrOfRows += 1; +} catch (DataFileWriter.AppendWriteException awe) { +Throwable rootCause = ExceptionUtils.getRootCause(awe); +if(rootCause instanceof UnresolvedUnionException) { +UnresolvedUnionException uue = (UnresolvedUnionException) rootCause; +throw new RuntimeException( +"Unable to resolve union for value " + uue.getUnresolvedDatum() + +" with type " + uue.getUnresolvedDatum().getClass().getCanonicalName() + +" while appending record " + rec, +awe); +} else { +throw awe; +} +} if (options.maxRows > 0 && nrOfRows == options.maxRows) break;
nifi git commit: NIFI-5891 fix handling of null logical types in Hive3Streaming processor
Repository: nifi Updated Branches: refs/heads/master 9a1ab4c50 -> c51512f5e NIFI-5891 fix handling of null logical types in Hive3Streaming processor NIFI-5891: Fixed Checkstyle issues Signed-off-by: Matthew Burgess This closes #3216 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/c51512f5 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/c51512f5 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/c51512f5 Branch: refs/heads/master Commit: c51512f5e33cbd413b1fda8700408aa95614680e Parents: 9a1ab4c Author: gkkorir Authored: Thu Dec 13 17:25:37 2018 +0300 Committer: Matthew Burgess Committed: Thu Dec 13 10:23:18 2018 -0500 -- .../apache/hive/streaming/NiFiRecordSerDe.java | 28 ++-- .../processors/hive/TestPutHive3Streaming.java | 140 +++ 2 files changed, 159 insertions(+), 9 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/c51512f5/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java -- diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java index 932772e..e628474 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java @@ -223,27 +223,37 @@ public class NiFiRecordSerDe extends AbstractSerDe { break; case DATE: Date d = record.getAsDate(fieldName, field.getDataType().getFormat()); -org.apache.hadoop.hive.common.type.Date hiveDate = new org.apache.hadoop.hive.common.type.Date(); -hiveDate.setTimeInMillis(d.getTime()); -val = hiveDate; +if(d != null) { +org.apache.hadoop.hive.common.type.Date hiveDate = new org.apache.hadoop.hive.common.type.Date(); +hiveDate.setTimeInMillis(d.getTime()); +val = hiveDate; +} else { +val = null; +} break; // ORC doesn't currently handle TIMESTAMPLOCALTZ case TIMESTAMP: Timestamp ts = DataTypeUtils.toTimestamp(record.getValue(fieldName), () -> DataTypeUtils.getDateFormat(field.getDataType().getFormat()), fieldName); -// Convert to Hive's Timestamp type -org.apache.hadoop.hive.common.type.Timestamp hivetimestamp = new org.apache.hadoop.hive.common.type.Timestamp(); -hivetimestamp.setTimeInMillis(ts.getTime(), ts.getNanos()); -val = hivetimestamp; +if(ts != null) { +// Convert to Hive's Timestamp type +org.apache.hadoop.hive.common.type.Timestamp hivetimestamp = new org.apache.hadoop.hive.common.type.Timestamp(); +hivetimestamp.setTimeInMillis(ts.getTime(), ts.getNanos()); +val = hivetimestamp; +} else { +val = null; +} break; case DECIMAL: -val = HiveDecimal.create(record.getAsDouble(fieldName)); +Double value = record.getAsDouble(fieldName); +val = value == null ? null : HiveDecimal.create(value); break; default: throw new IllegalArgumentException("Field " + fieldName + " cannot be converted to type: " + primitiveCategory.name()); } break; case LIST: -val = Arrays.asList(record.getAsArray(fieldName)); +Object[] value = record.getAsArray(fieldName); +val = value == null ? null : Arrays.asList(value); break; case MAP: val = record.getValue(fieldName); http://git-wip-us.apache.org/repos/asf/nifi/blob/c51512f5/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java -- diff --git
nifi git commit: NIFI-4579: Fix ValidateRecord type coercing
Repository: nifi Updated Branches: refs/heads/master c51512f5e -> 0efddf47d NIFI-4579: Fix ValidateRecord type coercing Signed-off-by: Matthew Burgess This closes #2794 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/0efddf47 Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/0efddf47 Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/0efddf47 Branch: refs/heads/master Commit: 0efddf47d516b62d7d9c61142d20ce40bcec675f Parents: c51512f Author: Koji Kawamura Authored: Thu Jun 14 15:39:17 2018 +0900 Committer: Matthew Burgess Committed: Thu Dec 13 16:10:41 2018 -0500 -- .../processors/standard/ValidateRecord.java | 18 +- .../processors/standard/TestValidateRecord.java | 223 +++ 2 files changed, 234 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/0efddf47/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java -- diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java index 52f462a..bc39ecf 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ValidateRecord.java @@ -109,7 +109,9 @@ public class ValidateRecord extends AbstractProcessor { static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() .name("record-writer") .displayName("Record Writer") -.description("Specifies the Controller Service to use for writing out the records") +.description("Specifies the Controller Service to use for writing out the records. " ++ "Regardless of the Controller Service schema access configuration, " ++ "the schema that is used to validate record is used to write the valid results.") .identifiesControllerService(RecordSetWriterFactory.class) .required(true) .build(); @@ -117,7 +119,8 @@ public class ValidateRecord extends AbstractProcessor { .name("invalid-record-writer") .displayName("Record Writer for Invalid Records") .description("If specified, this Controller Service will be used to write out any records that are invalid. " -+ "If not specified, the writer specified by the \"Record Writer\" property will be used. This is useful, for example, when the configured " ++ "If not specified, the writer specified by the \"Record Writer\" property will be used with the schema used to read the input records. " ++ "This is useful, for example, when the configured " + "Record Writer cannot write data that does not adhere to its schema (as is the case with Avro) or when it is desirable to keep invalid records " + "in their original format while converting valid records to another format.") .identifiesControllerService(RecordSetWriterFactory.class) @@ -161,7 +164,7 @@ public class ValidateRecord extends AbstractProcessor { .displayName("Allow Extra Fields") .description("If the incoming data has fields that are not present in the schema, this property determines whether or not the Record is valid. " + "If true, the Record is still valid. If false, the Record will be invalid due to the extra fields.") -.expressionLanguageSupported(false) +.expressionLanguageSupported(ExpressionLanguageScope.NONE) .allowableValues("true", "false") .defaultValue("true") .required(true) @@ -172,7 +175,7 @@ public class ValidateRecord extends AbstractProcessor { .description("If the incoming data has a Record where a field is not of the correct type, this property determine whether how to handle the Record. " + "If true, the Record will still be considered invalid. If false, the Record will be considered valid and the field will be coerced into the " + "correct type (if possible, according to the type coercion supported by the Record Writer).") -.expressionLanguageSupported(false) +.expressionLanguageSupported(ExpressionLanguageScope.NONE) .allowableValues("true", "false") .defaultValue("true") .required(true) @@ -292,7 +295,8 @@ public class ValidateRecord extends AbstractProcessor { validFlowFile =
nifi git commit: NIFI-5843 added subjects to the error message when neither script body nor script text are provided
Repository: nifi Updated Branches: refs/heads/master 455e3c1bc -> 98cdb50cc NIFI-5843 added subjects to the error message when neither script body nor script text are provided Signed-off-by: Matthew Burgess This closes #3186 Project: http://git-wip-us.apache.org/repos/asf/nifi/repo Commit: http://git-wip-us.apache.org/repos/asf/nifi/commit/98cdb50c Tree: http://git-wip-us.apache.org/repos/asf/nifi/tree/98cdb50c Diff: http://git-wip-us.apache.org/repos/asf/nifi/diff/98cdb50c Branch: refs/heads/master Commit: 98cdb50cc0a72957959188d352b9507d4f85dea1 Parents: 455e3c1 Author: Alex Savitsky Authored: Wed Nov 28 10:10:08 2018 -0500 Committer: Matthew Burgess Committed: Wed Nov 28 11:13:17 2018 -0500 -- .../java/org/apache/nifi/script/ScriptingComponentHelper.java| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/nifi/blob/98cdb50c/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java -- diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java index 776fc49..831d305 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java @@ -127,8 +127,8 @@ public class ScriptingComponentHelper { // Verify that exactly one of "script file" or "script body" is set Map propertyMap = validationContext.getProperties(); if (StringUtils.isEmpty(propertyMap.get(ScriptingComponentUtils.SCRIPT_FILE)) == StringUtils.isEmpty(propertyMap.get(ScriptingComponentUtils.SCRIPT_BODY))) { -results.add(new ValidationResult.Builder().valid(false).explanation( -"Exactly one of Script File or Script Body must be set").build()); +results.add(new ValidationResult.Builder().subject("Script Body or Script File").valid(false).explanation( +"exactly one of Script File or Script Body must be set").build()); } return results;
[nifi] branch master updated: NIFI-5945 Add support for password login to kerberos code in nifi-security-utils
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 2bbfb32 NIFI-5945 Add support for password login to kerberos code in nifi-security-utils 2bbfb32 is described below commit 2bbfb3217be40abe4af7ddb8627808d12d99bb17 Author: Bryan Bende AuthorDate: Wed Jan 9 17:37:10 2019 -0500 NIFI-5945 Add support for password login to kerberos code in nifi-security-utils Fixing solr test Signed-off-by: Matthew Burgess This closes #3256 --- ...rdKeytabUser.java => AbstractKerberosUser.java} | 41 +++- .../nifi/security/krb/ConfigurationUtil.java | 25 + .../krb/{KeytabAction.java => KerberosAction.java} | 34 +++ .../nifi/security/krb/KerberosKeytabUser.java | 59 +++ .../nifi/security/krb/KerberosPasswordUser.java| 110 + .../krb/{KeytabUser.java => KerberosUser.java} | 7 +- .../nifi/security/krb/KeytabConfiguration.java | 9 +- .../org/apache/nifi/security/krb/KDCServer.java| 5 +- .../krb/{KeytabUserIT.java => KerberosUserIT.java} | 63 +--- .../nifi/security/krb/TestKeytabConfiguration.java | 2 +- .../apache/nifi/processors/solr/SolrProcessor.java | 32 +++--- .../processors/solr/TestPutSolrContentStream.java | 45 - 12 files changed, 320 insertions(+), 112 deletions(-) diff --git a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/StandardKeytabUser.java b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/AbstractKerberosUser.java similarity index 86% rename from nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/StandardKeytabUser.java rename to nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/AbstractKerberosUser.java index 7302ee0..32eb9bb 100644 --- a/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/StandardKeytabUser.java +++ b/nifi-commons/nifi-security-utils/src/main/java/org/apache/nifi/security/krb/AbstractKerberosUser.java @@ -23,7 +23,6 @@ import org.slf4j.LoggerFactory; import javax.security.auth.Subject; import javax.security.auth.kerberos.KerberosPrincipal; import javax.security.auth.kerberos.KerberosTicket; -import javax.security.auth.login.Configuration; import javax.security.auth.login.LoginContext; import javax.security.auth.login.LoginException; import java.security.PrivilegedAction; @@ -34,14 +33,9 @@ import java.util.Date; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; -/** - * Used to authenticate and execute actions when Kerberos is enabled and a keytab is being used. - * - * Some of the functionality in this class is adapted from Hadoop's UserGroupInformation. - */ -public class StandardKeytabUser implements KeytabUser { +public abstract class AbstractKerberosUser implements KerberosUser { -private static final Logger LOGGER = LoggerFactory.getLogger(StandardKeytabUser.class); +private static final Logger LOGGER = LoggerFactory.getLogger(AbstractKerberosUser.class); static final String DATE_FORMAT = "-MM-dd'T'HH:mm:ss'Z'"; @@ -50,18 +44,15 @@ public class StandardKeytabUser implements KeytabUser { */ static final float TICKET_RENEW_WINDOW = 0.80f; -private final String principal; -private final String keytabFile; -private final AtomicBoolean loggedIn = new AtomicBoolean(false); +protected final String principal; +protected final AtomicBoolean loggedIn = new AtomicBoolean(false); -private Subject subject; -private LoginContext loginContext; +protected Subject subject; +protected LoginContext loginContext; -public StandardKeytabUser(final String principal, final String keytabFile) { +public AbstractKerberosUser(final String principal) { this.principal = principal; -this.keytabFile = keytabFile; -Validate.notBlank(principal); -Validate.notBlank(keytabFile); +Validate.notBlank(this.principal); } /** @@ -80,19 +71,19 @@ public class StandardKeytabUser implements KeytabUser { if (loginContext == null) { LOGGER.debug("Initializing new login context..."); this.subject = new Subject(); - -final Configuration config = new KeytabConfiguration(principal, keytabFile); -this.loginContext = new LoginContext("KeytabConf", subject, null, config); +this.loginContext = createLoginContext(subject); } loginContext.login(); loggedIn.set(true); LOGGER.debug("Successful login for {}", new Object[]{principal}); } catch (LoginException le) { -throw ne
[nifi] branch master updated: NIFI-5879: Fixed bug in FileSystemRepository that can occur if an InputStream is obtained, then more data is written to the Content Claim - the InputStream would end befo
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new cf41c10 NIFI-5879: Fixed bug in FileSystemRepository that can occur if an InputStream is obtained, then more data is written to the Content Claim - the InputStream would end before allowing the sequential data to be read. Also fixed bugs in LimitedInputStream related to available(), mark(), and reset() and the corresponding unit tests. Additionally, found that one call to StandardProcessSession.read() was not properly flushing the output of any Content Claim that has been writte [...] cf41c10 is described below commit cf41c10546d940aa86d0287bbeb2cdaf4a6c8a2a Author: Mark Payne AuthorDate: Thu Dec 6 16:22:29 2018 -0500 NIFI-5879: Fixed bug in FileSystemRepository that can occur if an InputStream is obtained, then more data is written to the Content Claim - the InputStream would end before allowing the sequential data to be read. Also fixed bugs in LimitedInputStream related to available(), mark(), and reset() and the corresponding unit tests. Additionally, found that one call to StandardProcessSession.read() was not properly flushing the output of any Content Claim that has been written to before at [...] Signed-off-by: Matthew Burgess This closes #3207 --- .../repository/FileSystemRepository.java | 11 +++- .../repository/StandardProcessSession.java | 4 +- .../repository/io/LimitedInputStream.java | 34 --- .../repository/TestFileSystemRepository.java | 69 ++ .../repository/io/TestLimitedInputStream.java | 17 +++--- 5 files changed, 94 insertions(+), 41 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java index c041f5c..125cd50 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/FileSystemRepository.java @@ -864,9 +864,16 @@ public class FileSystemRepository implements ContentRepository { } -// see javadocs for claim.getLength() as to why we do this. +// A claim length of -1 indicates that the claim is still being written to and we don't know +// the length. In this case, we don't limit the Input Stream. If the Length has been populated, though, +// it is possible that the Length could then be extended. However, we do want to avoid ever allowing the +// stream to read past the end of the Content Claim. To accomplish this, we use a LimitedInputStream but +// provide a LongSupplier for the length instead of a Long value. this allows us to continue reading until +// we get to the end of the Claim, even if the Claim grows. This may happen, for instance, if we obtain an +// InputStream for this claim, then read from it, write more to the claim, and then attempt to read again. In +// such a case, since we have written to that same Claim, we should still be able to read those bytes. if (claim.getLength() >= 0) { -return new LimitedInputStream(fis, claim.getLength()); +return new LimitedInputStream(fis, claim::getLength); } else { return fis; } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java index 4354dc4..cc3ac19 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/StandardProcessSession.java @@ -2267,7 +2267,9 @@ public final class StandardProcessSession implements ProcessSession, ProvenanceE final StandardRepositoryRecord record = getRecord(source); try { -ensureNotAppending(record.getCurrentClaim()); +final ContentClaim currentClaim = record.getCurrentClaim(); +ensureNotAppending(currentClaim); +claimCache.flush(currentClaim); } catch (final IOException e) { throw
[nifi] branch master updated: NIFI-5854 Added skeleton logic to convert decimal time units. Added helper methods. Added unit tests.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new b59fa5a NIFI-5854 Added skeleton logic to convert decimal time units. Added helper methods. Added unit tests. b59fa5a is described below commit b59fa5af1f3232581e1b3903e3e2f408d9daa323 Author: Andy LoPresto AuthorDate: Thu Nov 29 20:17:58 2018 -0800 NIFI-5854 Added skeleton logic to convert decimal time units. Added helper methods. Added unit tests. NIFI-5854 [WIP] Cleaned up logic. Resolved failing unit tests due to error message change. NIFI-5854 [WIP] All helper method unit tests pass. NIFI-5854 [WIP] FormatUtils#getPreciseTimeDuration() now handles all tested inputs correctly. Added unit tests. NIFI-5854 [WIP] FormatUtils#getTimeDuration() still using long. Added unit tests. Renamed existing unit tests to reflect method under test. NIFI-5854 FormatUtils#getTimeDuration() returns long but now accepts decimal inputs. Added @Deprecation warnings (will update callers where possible). All unit tests pass. NIFI-5854 Fixed unit tests (ran in IDE but not Maven) due to int overflows. Fixed checkstyle issues. NIFI-5854 Fixed typo in Javadoc. NIFI-5854 Fixed typo in Javadoc. Signed-off-by: Matthew Burgess This closes #3193 --- .../java/org/apache/nifi/util/FormatUtils.java | 227 -- .../nifi/processor/TestFormatUtilsGroovy.groovy| 130 -- .../apache/nifi/util/TestFormatUtilsGroovy.groovy | 488 + 3 files changed, 685 insertions(+), 160 deletions(-) diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java index 1c9140b..7d2992f 100644 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java +++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/util/FormatUtils.java @@ -17,12 +17,13 @@ package org.apache.nifi.util; import java.text.NumberFormat; +import java.util.Arrays; +import java.util.List; import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; public class FormatUtils { - private static final String UNION = "|"; // for Data Sizes @@ -41,8 +42,9 @@ public class FormatUtils { private static final String WEEKS = join(UNION, "w", "wk", "wks", "week", "weeks"); private static final String VALID_TIME_UNITS = join(UNION, NANOS, MILLIS, SECS, MINS, HOURS, DAYS, WEEKS); -public static final String TIME_DURATION_REGEX = "(\\d+)\\s*(" + VALID_TIME_UNITS + ")"; +public static final String TIME_DURATION_REGEX = "([\\d.]+)\\s*(" + VALID_TIME_UNITS + ")"; public static final Pattern TIME_DURATION_PATTERN = Pattern.compile(TIME_DURATION_REGEX); +private static final List TIME_UNIT_MULTIPLIERS = Arrays.asList(1000L, 1000L, 1000L, 60L, 60L, 24L); /** * Formats the specified count by adding commas. @@ -58,7 +60,7 @@ public class FormatUtils { * Formats the specified duration in 'mm:ss.SSS' format. * * @param sourceDuration the duration to format - * @param sourceUnit the unit to interpret the duration + * @param sourceUnit the unit to interpret the duration * @return representation of the given time data in minutes/seconds */ public static String formatMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) { @@ -79,7 +81,7 @@ public class FormatUtils { * Formats the specified duration in 'HH:mm:ss.SSS' format. * * @param sourceDuration the duration to format - * @param sourceUnit the unit to interpret the duration + * @param sourceUnit the unit to interpret the duration * @return representation of the given time data in hours/minutes/seconds */ public static String formatHoursMinutesSeconds(final long sourceDuration, final TimeUnit sourceUnit) { @@ -139,65 +141,230 @@ public class FormatUtils { return format.format(dataSize) + " bytes"; } +/** + * Returns a time duration in the requested {@link TimeUnit} after parsing the {@code String} + * input. If the resulting value is a decimal (i.e. + * {@code 25 hours -> TimeUnit.DAYS = 1.04}), the value is rounded. + * + * @param value the raw String input (i.e. "28 minutes") + * @param desiredUnit the requested output {@link TimeUnit} + * @return the whole number value of this duration in the requested units + * @deprecated As of Apache NiFi 1.9.0, because this method only returns whole numbers, use {@link #getPreciseTim
[nifi] 01/02: NIFI-6117: Fix BIGINT handling in DataTypeUtils
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit c2dc0910b76565817e95fa349e56aca5d8f2328e Author: Matthew Burgess AuthorDate: Wed Mar 13 18:35:38 2019 -0400 NIFI-6117: Fix BIGINT handling in DataTypeUtils --- .../serialization/record/util/DataTypeUtils.java | 20 +- .../serialization/record/TestDataTypeUtils.java| 31 ++ 2 files changed, 50 insertions(+), 1 deletion(-) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index 63db142..c6dcb26 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -1083,12 +1083,30 @@ public class DataTypeUtils { if (value instanceof Long) { return BigInteger.valueOf((Long) value); } +if (value instanceof Integer) { +return BigInteger.valueOf(((Integer) value).longValue()); +} +if (value instanceof Short) { +return BigInteger.valueOf(((Short) value).longValue()); +} +if (value instanceof String) { +try { +return new BigInteger((String) value); +} catch (NumberFormatException nfe) { +throw new IllegalTypeConversionException("Cannot convert value [" + value + "] of type " + value.getClass() + " to BigInteger for field " + fieldName ++ ", value is not a valid representation of BigInteger", nfe); +} +} 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); +return value instanceof BigInteger +|| value instanceof Long +|| value instanceof Integer +|| value instanceof Short +|| value instanceof String; } public static Boolean toBoolean(final Object value, final String fieldName) { diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java index cef0eec..c64ed37 100644 --- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java @@ -19,8 +19,10 @@ package org.apache.nifi.serialization.record; import org.apache.nifi.serialization.SimpleRecordSchema; import org.apache.nifi.serialization.record.util.DataTypeUtils; +import org.apache.nifi.serialization.record.util.IllegalTypeConversionException; import org.junit.Test; +import java.math.BigInteger; import java.nio.charset.StandardCharsets; import java.sql.Timestamp; import java.util.ArrayList; @@ -287,4 +289,33 @@ public class TestDataTypeUtils { testMap.put("Hello", "World"); assertTrue(DataTypeUtils.isCompatibleDataType(testMap, RecordFieldType.RECORD.getDataType())); } + +@Test +public void testIsCompatibleDataTypeBigint() { +assertTrue(DataTypeUtils.isCompatibleDataType(new BigInteger("12345678901234567890"), RecordFieldType.BIGINT.getDataType())); +assertTrue(DataTypeUtils.isCompatibleDataType(1234567890123456789L, RecordFieldType.BIGINT.getDataType())); +assertTrue(DataTypeUtils.isCompatibleDataType(1, RecordFieldType.BIGINT.getDataType())); +assertTrue(DataTypeUtils.isCompatibleDataType((short) 1, RecordFieldType.BIGINT.getDataType())); +assertTrue(DataTypeUtils.isCompatibleDataType("12345678901234567890", RecordFieldType.BIGINT.getDataType())); +assertTrue(DataTypeUtils.isCompatibleDataType("1234567XYZ", RecordFieldType.BIGINT.getDataType())); // Compatible but the value might not be a valid BigInteger +assertFalse(DataTypeUtils.isCompatibleDataType(3.0f, RecordFieldType.BIGINT.getDataType())); +assertFalse(DataTypeUtils.isCompatibleDataType(3.0, RecordFieldType.BIGINT.getDataType())); +assertFalse(DataTypeUtils.isCompatibleDataType(new Long[]{1L, 2L}, RecordFieldType.BIGINT.getDataType())); +} + +@Test +public void testConvertDataTypeBigint() { +assertTrue(DataTypeUtils.convertType(new
[nifi] branch master updated (ead6a3b -> cae01d5)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from ead6a3b NIFI-6144 - Enable EL on listening port in HandleHttpRequest new c2dc091 NIFI-6117: Fix BIGINT handling in DataTypeUtils new cae01d5 NIFI-6117: Fix BIGINT handling in DataTypeUtils The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../serialization/record/util/DataTypeUtils.java | 34 +--- .../serialization/record/TestDataTypeUtils.java| 37 ++ 2 files changed, 66 insertions(+), 5 deletions(-)
[nifi] 02/02: NIFI-6117: Fix BIGINT handling in DataTypeUtils
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit cae01d555cc2f30b2a302b9dc8ec84db2a04469c Author: Koji Kawamura AuthorDate: Tue Mar 26 08:37:20 2019 +0900 NIFI-6117: Fix BIGINT handling in DataTypeUtils Follow numeric type conversion convention used for other integral types. Signed-off-by: Matthew Burgess This closes #3371 --- .../serialization/record/util/DataTypeUtils.java | 36 +- .../serialization/record/TestDataTypeUtils.java| 36 +- 2 files changed, 42 insertions(+), 30 deletions(-) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index c6dcb26..3411b13 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -1080,15 +1080,11 @@ public class DataTypeUtils { if (value instanceof BigInteger) { return (BigInteger) value; } -if (value instanceof Long) { -return BigInteger.valueOf((Long) value); -} -if (value instanceof Integer) { -return BigInteger.valueOf(((Integer) value).longValue()); -} -if (value instanceof Short) { -return BigInteger.valueOf(((Short) value).longValue()); + +if (value instanceof Number) { +return BigInteger.valueOf(((Number) value).longValue()); } + if (value instanceof String) { try { return new BigInteger((String) value); @@ -1102,11 +1098,7 @@ public class DataTypeUtils { } public static boolean isBigIntTypeCompatible(final Object value) { -return value instanceof BigInteger -|| value instanceof Long -|| value instanceof Integer -|| value instanceof Short -|| value instanceof String; +return isNumberTypeCompatible(value, DataTypeUtils::isIntegral); } public static Boolean toBoolean(final Object value, final String fieldName) { @@ -1277,7 +1269,10 @@ public class DataTypeUtils { return false; } -private static boolean isIntegral(final String value, final long minValue, final long maxValue) { +/** + * Check if the value is an integral. + */ +private static boolean isIntegral(final String value) { if (value == null || value.isEmpty()) { return false; } @@ -1298,6 +1293,18 @@ public class DataTypeUtils { } } +return true; +} + +/** + * Check if the value is an integral within a value range. + */ +private static boolean isIntegral(final String value, final long minValue, final long maxValue) { + +if (!isIntegral(value)) { +return false; +} + try { final long longValue = Long.parseLong(value); return longValue >= minValue && longValue <= maxValue; @@ -1307,7 +1314,6 @@ public class DataTypeUtils { } } - public static Integer toInteger(final Object value, final String fieldName) { if (value == null) { return null; diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java index c64ed37..89a0490 100644 --- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.function.Function; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -292,27 +293,32 @@ public class TestDataTypeUtils { @Test public void testIsCompatibleDataTypeBigint() { -assertTrue(DataTypeUtils.isCompatibleDataType(new BigInteger("12345678901234567890"), RecordFieldType.BIGINT.getDataType())); -assertTrue(DataTypeUtils.isCompatibleDataType(1234567890123456789L, RecordFieldType.BIGINT.getDataType())); -assertTrue(DataTypeUtils.isCompatibleDataType(1, RecordFieldType.BIGINT.getDataType())); -assertTrue(DataTypeUtils.isCompatibleDataType((short) 1, RecordFieldType.BIGINT.getDataType())); -assertTrue(DataTypeUtils.isCompatibleDataType("12345678901234567890", RecordFieldType.BIGINT.getDataType())); -assertTrue(
[nifi] branch master updated: NIFI-6180: exposing firehose grace period to DruidTranquilityController
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3a63de2 NIFI-6180: exposing firehose grace period to DruidTranquilityController 3a63de2 is described below commit 3a63de2ae2debea95244bc5c424d8669d5833cd9 Author: Endre Zoltan Kovacs AuthorDate: Wed Apr 3 16:35:04 2019 +0200 NIFI-6180: exposing firehose grace period to DruidTranquilityController this allows for configuring `druidBeam.firehoseGracePeriod` (https://github.com/druid-io/tranquility/blob/master/docs/configuration.md#properties) NIFI-6180: Corrected typo in DruidTranquilityController Signed-off-by: Matthew Burgess This closes #3403 --- .../controller/druid/DruidTranquilityController.java | 18 -- .../druid/MockDruidTranquilityController.java | 2 +- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java index 0690264..78aab4c 100644 --- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java +++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java @@ -80,6 +80,7 @@ public class DruidTranquilityController extends AbstractControllerService implem private final static String FIREHOSE_PATTERN = "druid:firehose:%s"; private final static AllowableValue PT1M = new AllowableValue("PT1M", "1 minute", "1 minute"); +private final static AllowableValue PT5M = new AllowableValue("PT5M", "5 minutes", "5 minutes"); private final static AllowableValue PT10M = new AllowableValue("PT10M", "10 minutes", "10 minutes"); private final static AllowableValue PT60M = new AllowableValue("PT60M", "60 minutes", "1 hour"); @@ -276,6 +277,16 @@ public class DruidTranquilityController extends AbstractControllerService implem .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); +public static final PropertyDescriptor FIREHOSE_GRACE_PERIOD = new PropertyDescriptor.Builder() +.name("druid-cs-firehose-grace-period") +.displayName("Firehose Grace Period") +.description("An additional grace period, after the \"Late Event Grace Period\" (window period) has elapsed, but before the indexing task is shut down.") +.required(true) +.allowableValues(PT1M, PT5M, PT10M, PT60M) +.defaultValue(PT5M.getValue()) +.addValidator(StandardValidators.NON_EMPTY_VALIDATOR) +.build(); + public static final PropertyDescriptor MAX_BATCH_SIZE = new PropertyDescriptor.Builder() .name("druid-cs-batch-size") .displayName("Batch Size") @@ -332,6 +343,7 @@ public class DruidTranquilityController extends AbstractControllerService implem props.add(QUERY_GRANULARITY); props.add(INDEX_RETRY_PERIOD); props.add(WINDOW_PERIOD); +props.add(FIREHOSE_GRACE_PERIOD); props.add(TIMESTAMP_FIELD); props.add(MAX_BATCH_SIZE); props.add(MAX_PENDING_BATCHES); @@ -378,6 +390,7 @@ public class DruidTranquilityController extends AbstractControllerService implem final String segmentGranularity = context.getProperty(SEGMENT_GRANULARITY).getValue(); final String queryGranularity = context.getProperty(QUERY_GRANULARITY).getValue(); final String windowPeriod = context.getProperty(WINDOW_PERIOD).getValue(); +final String firehoseGracePeriod = context.getProperty(FIREHOSE_GRACE_PERIOD).getValue(); final String indexRetryPeriod = context.getProperty(INDEX_RETRY_PERIOD).getValue(); final String aggregatorJSON = context.getProperty(AGGREGATOR_JSON).evaluateAttributeExpressions().getValue(); final String dimensionsStringList = context.getProperty(DIMENSIONS_LIST).evaluateAttributeExpressions().getValue(); @@ -416,7 +429,7 @@ public class DruidTranquilityController extends AbstractControllerService implem final TimestampSpec timestampSpec = new TimestampSpec(timestampField, "auto", null); final Beam> beam = buildBeam(dataSource, indexService, discoveryPath, clusterPartitions, clusterReplication, -segmentGra
[nifi] branch master updated: NIFI-6179: minor changes to DruidTranquilityController
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e907b68 NIFI-6179: minor changes to DruidTranquilityController e907b68 is described below commit e907b689e8ea4ae9deb30812d43d66a23440b6ae Author: Endre Zoltan Kovacs AuthorDate: Wed Apr 3 16:09:35 2019 +0200 NIFI-6179: minor changes to DruidTranquilityController - hooking in property INDEX_RETRY_PERIOD to list of properties - fixing property description Signed-off-by: Matthew Burgess This closes #3402 --- .../org/apache/nifi/controller/druid/DruidTranquilityController.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java index 5d617f7..0690264 100644 --- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java +++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java @@ -259,7 +259,7 @@ public class DruidTranquilityController extends AbstractControllerService implem public static final PropertyDescriptor INDEX_RETRY_PERIOD = new PropertyDescriptor.Builder() .name("druid-cs-index-retry-period") .displayName("Index Retry Period") -.description("Grace period to allow late arriving events for real time ingest.") +.description("Time period, until a transiently failing indexing service overlord call is retried, before giving up.") .required(true) .allowableValues(PT1M, PT10M, PT60M) .defaultValue(PT10M.getValue()) @@ -330,6 +330,7 @@ public class DruidTranquilityController extends AbstractControllerService implem props.add(AGGREGATOR_JSON); props.add(SEGMENT_GRANULARITY); props.add(QUERY_GRANULARITY); +props.add(INDEX_RETRY_PERIOD); props.add(WINDOW_PERIOD); props.add(TIMESTAMP_FIELD); props.add(MAX_BATCH_SIZE);
[nifi] branch master updated: NIFI-6016 PutCassandraRecord batch size
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new d36fa6a NIFI-6016 PutCassandraRecord batch size d36fa6a is described below commit d36fa6a210261c147453144ef2fb202745319c84 Author: dnsbtchr AuthorDate: Tue Feb 26 11:47:17 2019 +0100 NIFI-6016 PutCassandraRecord batch size Adds resetting the batch size to fix broken batch processing Removes empty line Signed-off-by: Matthew Burgess This closes #3337 --- .../java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java| 1 + 1 file changed, 1 insertion(+) diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java index 84016dc..8acc36a 100644 --- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java +++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java @@ -167,6 +167,7 @@ public class PutCassandraRecord extends AbstractCassandraProcessor { if (recordsAdded.incrementAndGet() == batchSize) { connectionSession.execute(batchStatement); batchStatement.clear(); +recordsAdded.set(0); } }
[nifi] branch master updated: NIFI-6186 Resolve handling of module paths in JythonScriptEngineConfigurator
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new c93ee5a NIFI-6186 Resolve handling of module paths in JythonScriptEngineConfigurator c93ee5a is described below commit c93ee5ad5949858bd3f4eec2067ae30016ea76ac Author: ambah AuthorDate: Wed Apr 3 18:15:58 2019 -0400 NIFI-6186 Resolve handling of module paths in JythonScriptEngineConfigurator Signed-off-by: Matthew Burgess This closes #3406 --- .../impl/JythonScriptEngineConfigurator.java | 3 +- .../nifi/processors/script/TestInvokeJython.java | 55 .../src/test/resources/jython/test_modules_path.py | 74 ++ 3 files changed, 131 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/JythonScriptEngineConfigurator.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/JythonScriptEngineConfigurator.java index 458658b..3b8271b 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/JythonScriptEngineConfigurator.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/JythonScriptEngineConfigurator.java @@ -18,6 +18,7 @@ package org.apache.nifi.script.impl; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processors.script.ScriptEngineConfigurator; +import org.python.core.PyString; import javax.script.ScriptEngine; import javax.script.ScriptException; @@ -47,7 +48,7 @@ public class JythonScriptEngineConfigurator implements ScriptEngineConfigurator engine.eval("import sys"); if (modulePaths != null) { for (String modulePath : modulePaths) { -engine.eval("sys.path.append('" + modulePath + "')"); +engine.eval("sys.path.append(" + PyString.encode_UnicodeEscape(modulePath, true) + ")"); } } } diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java index 5b17e04..345f4af 100755 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJython.java @@ -16,6 +16,7 @@ */ package org.apache.nifi.processors.script; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.components.ValidationResult; import org.apache.nifi.script.ScriptingComponentUtils; import org.apache.nifi.util.MockFlowFile; @@ -31,6 +32,9 @@ import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Arrays; +import java.util.stream.Collectors; +import java.util.stream.IntStream; public class TestInvokeJython extends BaseScriptTest { @@ -95,6 +99,57 @@ public class TestInvokeJython extends BaseScriptTest { } /** + * Test a script that has a Jython processor that reads the system path as controlled by the Module Directory property then stores it in the attributes of the flowfile being routed. + * + * This tests whether the JythonScriptEngineConfigurator successfully translates the "Module Directory" property into Python system paths, even with strings that contain Python escape sequences + * + * @throws Exception Any error encountered while testing + */ +@Test +public void testUpdateAttributeFromProcessorModulePaths() throws Exception { +// Prepare a set of easily identified paths for the Module Directory property +final String moduleDirectoryTestPrefix = "test"; +final String[] testModuleDirectoryValues = { "abc","\\a\\b\\c","\\123","\\d\"e" }; +final int numTestValues = testModuleDirectoryValues.length; +// Prepend each module directory value with a simple prefix and an identifying number so we can identify it later. +final List testModuleDirectoryFullValues = IntStream.range(0,numTestValues) +.boxed() +.map(i -> String.format("%s#%s#%s",moduleDirectoryTestPrefix,i,testModuleDirectoryValues[i])) +.collect(Collectors.toList()); +final String testModuleDirectoryCombined = String.join(","
[nifi] branch master updated: NIFI-6172 Fixed a bug that caused ElasticSearchStringLookupService to always return a value even when it should return Optional.empty().
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 4810f6d NIFI-6172 Fixed a bug that caused ElasticSearchStringLookupService to always return a value even when it should return Optional.empty(). 4810f6d is described below commit 4810f6d32e7f30763b22c704a65603120e4b3c54 Author: Mike Thomsen AuthorDate: Tue Apr 2 10:32:36 2019 -0400 NIFI-6172 Fixed a bug that caused ElasticSearchStringLookupService to always return a value even when it should return Optional.empty(). NIFI-6172 Fixed broken integration test. Signed-off-by: Matthew Burgess This closes #3399 --- .../apache/nifi/elasticsearch/ElasticSearchStringLookupService.java | 6 +- .../elasticsearch/integration/ElasticSearchLookupService_IT.groovy | 2 +- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchStringLookupService.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchStringLookupService.java index 0ff9672..b833e79 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchStringLookupService.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchStringLookupService.java @@ -88,7 +88,11 @@ public class ElasticSearchStringLookupService extends AbstractControllerService try { final String id = (String) coordinates.get(ID); final Map enums = esClient.get(index, type, id); -return Optional.of(mapper.writeValueAsString(enums)); +if (enums == null) { +return Optional.empty(); +} else { +return Optional.ofNullable(mapper.writeValueAsString(enums)); +} } catch (IOException e) { throw new LookupFailureException(e); } diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/test/groovy/org/apache/nifi/elasticsearch/integration/ElasticSearchLookupService_IT.groovy b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/test/groovy/org/apache/nifi/elasticsearch/integration/ElasticSearchLookupService_IT.groovy index b255836..2ce9573 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/test/groovy/org/apache/nifi/elasticsearch/integration/ElasticSearchLookupService_IT.groovy +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/test/groovy/org/apache/nifi/elasticsearch/integration/ElasticSearchLookupService_IT.groovy @@ -203,7 +203,7 @@ class ElasticSearchLookupService_IT { def result = lookupService.lookup(coordinates) Assert.assertTrue(result.isPresent()) def rec = result.get() -["dateField": "2018-08-14T10:08:00Z", "longField": 15L].each { field -> +["dateField2": "2018-08-14T10:08:00Z", "longField2": 15L].each { field -> def value = rec.getValue(field.key) Assert.assertEquals(field.value, value) }
[nifi] branch master updated: NIFI-5172 Adding the ability to specify a record writer for PutElasticsearchHttpRecord in order to individually handle failed records
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new cd7edb1 NIFI-5172 Adding the ability to specify a record writer for PutElasticsearchHttpRecord in order to individually handle failed records cd7edb1 is described below commit cd7edb1c04fdd977de1fa30d1dbe4bf93c4afda2 Author: Joe Percivall AuthorDate: Sun Feb 10 19:47:31 2019 -0500 NIFI-5172 Adding the ability to specify a record writer for PutElasticsearchHttpRecord in order to individually handle failed records Addressing PR feedback Signed-off-by: Matthew Burgess This closes #3299 --- .../elasticsearch/PutElasticsearchHttpRecord.java | 129 +++-- .../TestPutElasticsearchHttpRecord.java| 106 +++-- 2 files changed, 214 insertions(+), 21 deletions(-) diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java index d431960..87dc5c3 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-processors/src/main/java/org/apache/nifi/processors/elasticsearch/PutElasticsearchHttpRecord.java @@ -55,6 +55,8 @@ 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.SimpleDateFormatValidator; import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.Record; @@ -72,6 +74,7 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.InputStream; +import java.io.OutputStream; import java.math.BigInteger; import java.net.URL; import java.nio.charset.Charset; @@ -121,6 +124,31 @@ public class PutElasticsearchHttpRecord extends AbstractElasticsearchHttpProcess .required(true) .build(); +static final PropertyDescriptor RECORD_WRITER = new PropertyDescriptor.Builder() +.name("put-es-record-record-writer") +.displayName("Record Writer") +.description("After sending a batch of records, Elasticsearch will report if individual records failed to insert. As an example, this can happen if the record doesn't match the mapping" + +"for the index it is being inserted into. This property specifies the Controller Service to use for writing out those individual records sent to 'failure'. If this is not set, " + +"then the whole FlowFile will be routed to failure (including any records which may have been inserted successfully). Note that this will only be used if Elasticsearch reports " + +"that individual records failed and that in the event that the entire FlowFile fails (e.g. in the event ES is down), the FF will be routed to failure without being interpreted " + +"by this record writer. If there is an error while attempting to route the failures, the entire FlowFile will be routed to Failure. Also if every record failed individually, " + +"the entire FlowFile will be routed to Failure without being parsed by the writer.") +.identifiesControllerService(RecordSetWriterFactory.class) +.required(false) +.build(); + +static final PropertyDescriptor LOG_ALL_ERRORS = new PropertyDescriptor.Builder() +.name("put-es-record-log-all-errors") +.displayName("Log all errors in batch") +.description("After sending a batch of records, Elasticsearch will report if individual records failed to insert. As an example, this can happen if the record doesn't match the mapping " + +"for the index it is being inserted into. If this is set to true, the processor will log the failure reason for the every failed record. When set to false only the first error " + +"in the batch will be logged.") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.required(false) +.defaultValue(&q
[nifi] branch master updated: Added tests for commons utilities StringUtils and FormatUtils These tests were written using Diffblue Cover.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 08a307c Added tests for commons utilities StringUtils and FormatUtils These tests were written using Diffblue Cover. 08a307c is described below commit 08a307cc6443de15713daa6ed29ed36a4b3e549f Author: Freddy Tuxworth AuthorDate: Wed Mar 6 12:27:00 2019 + Added tests for commons utilities StringUtils and FormatUtils These tests were written using Diffblue Cover. Fixed bug in FormatUtils.formatNanos Fix import asterisk warning Add Apache license header. Signed-off-by: Matthew Burgess This closes #3354 --- .../java/org/apache/nifi/util/StringUtilsTest.java | 100 + .../java/org/apache/nifi/util/FormatUtils.java | 4 +- .../org/apache/nifi/processor/TestFormatUtils.java | 50 ++- 3 files changed, 149 insertions(+), 5 deletions(-) diff --git a/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/StringUtilsTest.java b/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/StringUtilsTest.java new file mode 100644 index 000..5c689c6 --- /dev/null +++ b/nifi-commons/nifi-properties/src/test/java/org/apache/nifi/util/StringUtilsTest.java @@ -0,0 +1,100 @@ +/* + * 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.util; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import java.util.ArrayList; +import org.junit.Test; + +public class StringUtilsTest { + + @Test + public void testIsBlank() { +assertFalse(StringUtils.isBlank("0")); +assertFalse(StringUtils.isBlank("\u")); +assertFalse(StringUtils.isBlank(" \u ")); +assertFalse(StringUtils.isBlank(" \u5678 ")); + +assertTrue(StringUtils.isBlank(" ")); +assertTrue(StringUtils.isBlank("")); + } + + @Test + public void testStartsWith() { +assertFalse(StringUtils.startsWith("!", "something")); + +assertTrue(StringUtils.startsWith("!test", "")); +assertTrue(StringUtils.startsWith("!something", "!")); +assertTrue(StringUtils.startsWith(null, null)); + } + + @Test + public void testPadRight() { +assertEquals("sample", StringUtils.padRight("sample", 0, '0')); +assertEquals("sample", StringUtils.padRight("sample", 10, '0')); +assertEquals("00", StringUtils.padRight("", 10, '0')); + +assertNull(StringUtils.padRight(null, 0, '0')); + } + + @Test + public void testPadLeft() { +assertEquals("sample", StringUtils.padLeft("sample", 0, '0')); +assertEquals("sample", StringUtils.padLeft("sample", 10, '0')); +assertEquals("00", StringUtils.padLeft("", 10, '0')); + +assertNull(StringUtils.padLeft(null, 0, '0')); + } + + @Test + public void testIsEmpty() { +assertFalse(StringUtils.isEmpty("")); +assertFalse(StringUtils.isEmpty(" ")); + +assertTrue(StringUtils.isEmpty("")); +assertTrue(StringUtils.isEmpty(null)); + } + + @Test + public void testSubstringAfter() { +assertEquals("", StringUtils.substringAfter("", "")); +assertEquals("", StringUtils.substringAfter("", ">>")); +assertEquals("after", StringUtils.substringAfter("substring>>after", ">>")); +assertEquals("after>>another", StringUtils.substringAfter("substring>>after>>another", ">>")); +assertEquals("", StringUtils.substringAfter("substring>>after", null)); +assertEquals("", StringUtils
[nifi] 02/02: NIFI-6105: Fix handling of arrays of records/maps in record utilities
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit 2846d3c3c6e454f7960763bd69e36e879684a033 Author: Koji Kawamura AuthorDate: Wed Mar 13 14:34:16 2019 +0900 NIFI-6105: Fix handling of arrays of records/maps in record utilities Refactored to use the same check logic for Record and Map types Signed-off-by: Matthew Burgess This closes #3367 --- .../serialization/record/util/DataTypeUtils.java | 96 ++ 1 file changed, 42 insertions(+), 54 deletions(-) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index fb6cdbd..63db142 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -201,59 +201,8 @@ public class DataTypeUtils { case LONG: return isLongTypeCompatible(value); case RECORD: { -if (value == null) { -return false; -} - -// value may be a Map even when type is RECORD -if (value instanceof Map) { -final RecordSchema schema = ((RecordDataType) dataType).getChildSchema(); -if (schema == null) { -return true; -} -Map record = ((Map) value); -for (final RecordField childField : schema.getFields()) { -final Object childValue = record.get(childField.getFieldName()); -if (childValue == null && !childField.isNullable()) { -logger.debug("Value is not compatible with schema because field {} has a null value, which is not allowed in the schema", childField.getFieldName()); -return false; -} -if (childValue == null) { -continue; // consider compatible -} - -if (!isCompatibleDataType(childValue, childField.getDataType())) { -return false; -} -} -return true; -} -if (!(value instanceof Record)) { -return false; -} - final RecordSchema schema = ((RecordDataType) dataType).getChildSchema(); -if (schema == null) { -return true; -} - -final Record record = (Record) value; -for (final RecordField childField : schema.getFields()) { -final Object childValue = record.getValue(childField); -if (childValue == null && !childField.isNullable()) { -logger.debug("Value is not compatible with schema because field {} has a null value, which is not allowed in the schema", childField.getFieldName()); -return false; -} -if (childValue == null) { -continue; // consider compatible -} - -if (!isCompatibleDataType(childValue, childField.getDataType())) { -return false; -} -} - -return true; +return isRecordTypeCompatible(schema, value); } case SHORT: return isShortTypeCompatible(value); @@ -539,8 +488,47 @@ public class DataTypeUtils { return RecordFieldType.RECORD.getRecordDataType(schema); } -public static boolean isRecordTypeCompatible(final Object value) { -return value instanceof Record; +/** + * Check if the given record structured object compatible with the schema. + * @param schema record schema, schema validation will not be performed if schema is null + * @param value the record structured object, i.e. Record or Map + * @return True if the object is compatible with the schema + */ +private static boolean isRecordTypeCompatible(RecordSchema schema, Object value) { + +if (value == null) { +return false; +} + +if (!(value instanceof Record) && !(value instanceof Map)) { +return false; +} + +if (schema == null) { +return true; +} + +for (final RecordField childField : schema.getFields()) { +final Object childValue; +if (value instanceof Record) { +
[nifi] branch master updated (35d1cac -> 2846d3c)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 35d1cac Documentation : ConsumerKafka_2_0 - updated configuration details re… (#3360) new f91311d NIFI-6105: Fix handling of arrays of records/maps in record utilities new 2846d3c NIFI-6105: Fix handling of arrays of records/maps in record utilities The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: .../serialization/record/util/DataTypeUtils.java | 75 +- .../serialization/record/TestDataTypeUtils.java| 35 ++ 2 files changed, 80 insertions(+), 30 deletions(-)
[nifi] 01/02: NIFI-6105: Fix handling of arrays of records/maps in record utilities
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit f91311da9d759fb5712ee8121d54d27a7deea236 Author: Matthew Burgess AuthorDate: Wed Mar 6 09:53:13 2019 -0500 NIFI-6105: Fix handling of arrays of records/maps in record utilities This closes #3353. Signed-off-by: Koji Kawamura --- .../serialization/record/util/DataTypeUtils.java | 27 + .../serialization/record/TestDataTypeUtils.java| 35 ++ 2 files changed, 62 insertions(+) diff --git a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java index a399f67..fb6cdbd 100644 --- a/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java +++ b/nifi-commons/nifi-record/src/main/java/org/apache/nifi/serialization/record/util/DataTypeUtils.java @@ -204,6 +204,30 @@ public class DataTypeUtils { if (value == null) { return false; } + +// value may be a Map even when type is RECORD +if (value instanceof Map) { +final RecordSchema schema = ((RecordDataType) dataType).getChildSchema(); +if (schema == null) { +return true; +} +Map record = ((Map) value); +for (final RecordField childField : schema.getFields()) { +final Object childValue = record.get(childField.getFieldName()); +if (childValue == null && !childField.isNullable()) { +logger.debug("Value is not compatible with schema because field {} has a null value, which is not allowed in the schema", childField.getFieldName()); +return false; +} +if (childValue == null) { +continue; // consider compatible +} + +if (!isCompatibleDataType(childValue, childField.getDataType())) { +return false; +} +} +return true; +} if (!(value instanceof Record)) { return false; } @@ -687,6 +711,9 @@ public class DataTypeUtils { return convertRecordMapToJavaMap((Map) value, ((MapDataType) dataType).getValueType()); } else if (dataType != null && isScalarValue(dataType, value)) { return value; +} else if (value instanceof Object[] && dataType instanceof ArrayDataType) { +// This is likely a Map whose values are represented as an array. Return a new array with each element converted to a Java object +return convertRecordArrayToJavaArray((Object[]) value, ((ArrayDataType) dataType).getElementType()); } throw new IllegalTypeConversionException("Cannot convert value of class " + value.getClass().getName() + " because the type is not supported"); diff --git a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java index 45b65b4..cef0eec 100644 --- a/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java +++ b/nifi-commons/nifi-record/src/test/java/org/apache/nifi/serialization/record/TestDataTypeUtils.java @@ -108,6 +108,34 @@ public class TestDataTypeUtils { } @Test +public void testConvertArrayOfRecordsToJavaArray() { +final List fields = new ArrayList<>(); +fields.add(new RecordField("stringField", RecordFieldType.STRING.getDataType())); +fields.add(new RecordField("intField", RecordFieldType.INT.getDataType())); + +final RecordSchema schema = new SimpleRecordSchema(fields); + +final Map values1 = new HashMap<>(); +values1.put("stringField", "hello"); +values1.put("intField", 5); +final Record inputRecord1 = new MapRecord(schema, values1); + +final Map values2 = new HashMap<>(); +values2.put("stringField", "world"); +values2.put("intField", 50); +final Record inputRecord2 = new MapRecord(schema, values2); + +Object[] recordArray = {inputRecord1, inputRecord2}; +Object resultObj = DataTypeUtils.convertRecordFieldtoObject(recordArray, RecordFieldType.ARRAY.getArrayDataType(RecordFieldType.RECORD.get
[nifi] branch master updated: NIFI-6078: Create PostSlack processor
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 79e05c9 NIFI-6078: Create PostSlack processor 79e05c9 is described below commit 79e05c9f583aa1e4d390f380e375253bd7be198e Author: Peter Turcsanyi AuthorDate: Thu Feb 21 00:27:43 2019 +0100 NIFI-6078: Create PostSlack processor Processor for sending messages on Slack and optionally upload and attach the FlowFile content (e.g. an image) to the message. NIFI-6078: Remove username/icon properties. NIFI-6078: Make Text property optional. NIFI-6078: Documentation changes. Signed-off-by: Matthew Burgess This closes #3339 --- .../src/main/resources/META-INF/NOTICE | 42 ++ .../nifi-slack-processors/pom.xml | 10 + .../apache/nifi/processors/slack/PostSlack.java| 474 + .../services/org.apache.nifi.processor.Processor | 3 +- .../additionalDetails.html | 107 + .../processors/slack/PostSlackCaptureServlet.java | 103 + .../slack/PostSlackConfigValidationTest.java | 152 +++ .../processors/slack/PostSlackFileMessageTest.java | 312 ++ .../processors/slack/PostSlackTextMessageTest.java | 282 9 files changed, 1484 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-nar/src/main/resources/META-INF/NOTICE b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-nar/src/main/resources/META-INF/NOTICE index 53ea274..f3e9cb8 100644 --- a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-nar/src/main/resources/META-INF/NOTICE +++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-nar/src/main/resources/META-INF/NOTICE @@ -4,6 +4,48 @@ Copyright 2016-2019 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). +** +Apache Software License v2 +** + +The following binary components are provided under the Apache Software License v2 + +(ASLv2) Apache Commons Codec + The following NOTICE information applies: +Apache Commons Codec +Copyright 2002-2017 The Apache Software Foundation + +src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java +contains test data from http://aspell.net/test/orig/batch0.tab. +Copyright (C) 2002 Kevin Atkinson (kev...@gnu.org) + + === + +The content of package org.apache.commons.codec.language.bm has been translated +from the original php source code available at http://stevemorse.org/phoneticinfo.htm +with permission from the original authors. +Original source copyright: +Copyright (c) 2008 Alexander Beider & Stephen P. Morse. + +(ASLv2) Apache Commons Logging + The following NOTICE information applies: +Apache Commons Logging +Copyright 2003-2014 The Apache Software Foundation + +(ASLv2) Apache HttpComponents + The following NOTICE information applies: +Apache HttpClient +Copyright 1999-2019 The Apache Software Foundation + +Apache HttpCore +Copyright 2005-2019 The Apache Software Foundation + +Apache HttpMime +Copyright 1999-2019 The Apache Software Foundation + +This project contains annotations derived from JCIP-ANNOTATIONS +Copyright (c) 2005 Brian Goetz and Tim Peierls. See http://www.jcip.net + Common Development and Distribution License 1.1 diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml index a6aa3b5..eb10033 100644 --- a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml +++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/pom.xml @@ -37,6 +37,16 @@ 1.0.4 +org.apache.httpcomponents +httpclient +4.5.7 + + +org.apache.httpcomponents +httpmime +4.5.7 + + org.apache.nifi nifi-api diff --git a/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PostSlack.java b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PostSlack.java new file mode 100644 index 000..f581731 --- /dev/null +++ b/nifi-nar-bundles/nifi-slack-bundle/nifi-slack-processors/src/main/java/org/apache/nifi/processors/slack/PostSlack.java @@ -0,0 +1,474 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under
[nifi] branch master updated: NIFI-4735: ParseEVTX only outputs one event per chunk
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 48a6c81 NIFI-4735: ParseEVTX only outputs one event per chunk 48a6c81 is described below commit 48a6c81fa261339c645773a3124ce0ee351346c4 Author: Ferenc Szabó AuthorDate: Wed Mar 20 12:28:08 2019 +0100 NIFI-4735: ParseEVTX only outputs one event per chunk This change is based on https://github.com/apache/nifi/pull/2489 I have reproduced the issue with some additional test cases and test files then applied the original fix. commit message from the original change: Updated the EVTX FileHeader class to correctly check if there are more chunks in the file. Previously this would not process the last chunk. Updated the EVTX ChunkHeader class to correctly check if there are additional records in the chunk. Previously this would only process the first record of each chunk. It was using the fileLastRecordNumber where it should have been using the logLastRecordNumber value. Updated the EVTX unit tests to have the correct expected number of events and use the logLastRecordNumber. refactoring duplicated code and magic numbers Signed-off-by: Matthew Burgess This closes #2489 This closes #3379 --- .../nifi/processors/evtx/parser/ChunkHeader.java | 2 +- .../nifi/processors/evtx/parser/FileHeader.java| 4 +- .../apache/nifi/processors/evtx/ParseEvtxTest.java | 43 ++--- .../processors/evtx/parser/ChunkHeaderTest.java| 4 +- .../src/test/resources/1344_events.evtx| Bin 0 -> 1118208 bytes .../test/resources/3778_events_not_exported.evtx | Bin 0 -> 1052672 bytes 6 files changed, 43 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/ChunkHeader.java b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/ChunkHeader.java index 7f01adf..bb4e4d7 100644 --- a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/ChunkHeader.java +++ b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/ChunkHeader.java @@ -158,7 +158,7 @@ public class ChunkHeader extends Block { } public boolean hasNext() { -return fileLastRecordNumber.compareTo(recordNumber) > 0; +return logLastRecordNumber.compareTo(recordNumber) > 0; } public String getString(int offset) { diff --git a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/FileHeader.java b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/FileHeader.java index 8610fe9..914d518 100644 --- a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/FileHeader.java +++ b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/main/java/org/apache/nifi/processors/evtx/parser/FileHeader.java @@ -141,10 +141,10 @@ public class FileHeader extends Block { /** * Tests whether there are more chunks - * @return true iff there are chunks left + * @return true if there are chunks left */ public boolean hasNext() { -return count < chunkCount; +return count <= chunkCount; } /** diff --git a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/test/java/org/apache/nifi/processors/evtx/ParseEvtxTest.java b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/test/java/org/apache/nifi/processors/evtx/ParseEvtxTest.java index 2e5e90d..260869d 100644 --- a/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/test/java/org/apache/nifi/processors/evtx/ParseEvtxTest.java +++ b/nifi-nar-bundles/nifi-evtx-bundle/nifi-evtx-processors/src/test/java/org/apache/nifi/processors/evtx/ParseEvtxTest.java @@ -74,6 +74,7 @@ public class ParseEvtxTest { public static final String USER_DATA = "UserData"; public static final String EVENT_DATA = "EventData"; public static final Set DATA_TAGS = new HashSet<>(Arrays.asList(EVENT_DATA, USER_DATA)); +public static final int EXPECTED_SUCCESSFUL_EVENT_COUNT = 1053; @Mock FileHeaderFactory fileHeaderFactory; @@ -366,7 +367,7 @@ public class ParseEvtxTest { assertEquals(1, failureFlowFiles.size()); validateFlowFiles(failureFlowFiles); // We expect the same number of records to come out no matter the granularity -assertEquals(960, validateFlowFiles(failureFlowFiles)); +assertEquals(EXPECTED_SUCCESSFUL_EVENT_COUNT, validateFlowF
[nifi] branch master updated: NIFI-5983: handling parse problems in recordReader implementations
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 24a7d48 NIFI-5983: handling parse problems in recordReader implementations 24a7d48 is described below commit 24a7d480c8fde18a7dd64d7de80812d18eb2c5a4 Author: Endre Zoltan Kovacs AuthorDate: Thu Jan 31 14:47:21 2019 +0100 NIFI-5983: handling parse problems in recordReader implementations Fixed Checkstyle violation Signed-off-by: Matthew Burgess This closes #3282 --- .../nifi-record-serialization-services/pom.xml | 5 ++ .../org/apache/nifi/avro/AvroRecordReader.java | 25 +--- .../java/org/apache/nifi/csv/CSVRecordReader.java | 68 -- .../apache/nifi/csv/ITApacheCSVRecordReader.java | 35 ++- 4 files changed, 93 insertions(+), 40 deletions(-) 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 140a74b..b16464e 100755 --- 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 @@ -119,6 +119,11 @@ caffeine 2.6.2 + +com.google.guava +guava +27.0.1-jre + 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 c1d87b6..c9a624f 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 @@ -24,6 +24,8 @@ import org.apache.nifi.serialization.record.MapRecord; import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordSchema; +import com.google.common.base.Throwables; + import java.io.IOException; import java.util.Map; @@ -33,14 +35,21 @@ public abstract class AvroRecordReader implements RecordReader { @Override public Record nextRecord(final boolean coerceTypes, final boolean dropUnknownFields) throws IOException, MalformedRecordException { -GenericRecord record = nextAvroRecord(); -if (record == null) { -return null; +try { +GenericRecord record = nextAvroRecord(); +if (record == null) { +return null; +} + +final RecordSchema schema = getSchema(); +final Map values = AvroTypeUtil.convertAvroRecordToMap(record, schema); +return new MapRecord(schema, values); +} catch (IOException e) { +throw e; +} catch (MalformedRecordException e) { +throw e; +} catch (Exception e) { +throw new MalformedRecordException("Error while getting next record. Root cause: " + Throwables.getRootCause(e), e); } - -final RecordSchema schema = getSchema(); -final Map values = AvroTypeUtil.convertAvroRecordToMap(record, schema); -return new MapRecord(schema, values); } - } 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 299ad05..22a2e8a 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 @@ -42,6 +42,8 @@ import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; +import com.google.common.base.Throwables; + public class CSVRecordRead
[nifi] branch master updated: NIFI-5903: Allow RecordPath to be used in QueryRecord processor. Also some code cleanup and improvements to the docs
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 82a0434 NIFI-5903: Allow RecordPath to be used in QueryRecord processor. Also some code cleanup and improvements to the docs 82a0434 is described below commit 82a0434901c9c7d2f507f57f177073c3a18a30d4 Author: Mark Payne AuthorDate: Mon Dec 17 10:56:22 2018 -0500 NIFI-5903: Allow RecordPath to be used in QueryRecord processor. Also some code cleanup and improvements to the docs NIFI-5903: Removed TODO comments that were done NIFI-5903: Added support for working with MAP types to QueryRecord and associated RPATH functions Signed-off-by: Matthew Burgess This closes #3223 --- .../org/apache/nifi/record/path/RecordPathParser.g | 8 +- .../apache/nifi/record/path/TestRecordPath.java| 46 +- .../serialization/record/ResultSetRecordSet.java | 39 +- .../record/ArrayListRecordReader.java | 70 +++ .../record/ArrayListRecordWriter.java | 107 + .../nifi/processors/standard/QueryRecord.java | 288 +++- .../apache/nifi/queryrecord/FlowFileTableScan.java | 2 +- .../additionalDetails.html | 513 - .../nifi/processors/standard/TestQueryRecord.java | 452 +- 9 files changed, 1494 insertions(+), 31 deletions(-) diff --git a/nifi-commons/nifi-record-path/src/main/antlr3/org/apache/nifi/record/path/RecordPathParser.g b/nifi-commons/nifi-record-path/src/main/antlr3/org/apache/nifi/record/path/RecordPathParser.g index 5e406cb..682cd7e 100644 --- a/nifi-commons/nifi-record-path/src/main/antlr3/org/apache/nifi/record/path/RecordPathParser.g +++ b/nifi-commons/nifi-record-path/src/main/antlr3/org/apache/nifi/record/path/RecordPathParser.g @@ -172,8 +172,8 @@ notFunctionArgList : simpleFilterFunctionOrOperation -> notFilterFunction : NOT LPAREN notFunctionArgList RPAREN -> ^(FUNCTION NOT notFunctionArgList); - -filterFunction : simpleFilterFunction | notFilterFunction; + +filterFunction : simpleFilterFunction | notFilterFunction; @@ -200,11 +200,11 @@ selfReference : CHILD_SEPARATOR! CURRENT_FIELD; parentReference : CHILD_SEPARATOR RANGE -> ^(PARENT_REFERENCE); -nonSelfFieldRef : childReference | descendantReference | selfReference | parentReference; +nonSelfFieldRef : childReference | descendantReference | selfReference | parentReference | index; fieldRef : nonSelfFieldRef | CURRENT_FIELD; -subPath : fieldRef | index | predicate; +subPath : fieldRef | predicate; diff --git a/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java index 881fb64..7cc00e8 100644 --- a/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java +++ b/nifi-commons/nifi-record-path/src/test/java/org/apache/nifi/record/path/TestRecordPath.java @@ -17,9 +17,16 @@ package org.apache.nifi.record.path; -import static org.junit.Assert.assertArrayEquals; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import org.apache.nifi.record.path.exception.RecordPathException; +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 org.junit.Test; import java.nio.charset.IllegalCharsetNameException; import java.nio.charset.StandardCharsets; @@ -36,16 +43,9 @@ import java.util.Optional; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.apache.nifi.record.path.exception.RecordPathException; -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 org.junit.Test; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; public class TestRecordPath { @@ -255,6 +255,26 @@ public class TestRecordPath { } @Test +pu
[nifi] branch master updated: NIFI-5869 Support Reconnection for JMS
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3492313 NIFI-5869 Support Reconnection for JMS 3492313 is described below commit 3492313d0b3436cdd0f7390d46d403fed9d65b77 Author: Ed AuthorDate: Thu Jan 10 11:55:29 2019 -0500 NIFI-5869 Support Reconnection for JMS resets worker if it doesn't work anymore for any reason. this will add "reconnect" capabilities. Will solve issues for following use cases: - authentication changed after successful connection - JNDI mapping changed and requires recaching. - JMS server isn't available anymore or restarted. improved controller reset on exception Signed-off-by: Matthew Burgess This closes #3261 --- .../cf/JMSConnectionFactoryProviderDefinition.java | 8 + .../nifi/jms/cf/JMSConnectionFactoryProvider.java | 9 - .../jms/cf/JndiJmsConnectionFactoryProvider.java | 8 + .../nifi/jms/processors/AbstractJMSProcessor.java | 22 +++- .../org/apache/nifi/jms/processors/ConsumeJMS.java | 41 -- .../org/apache/nifi/jms/processors/JMSWorker.java | 9 + .../org/apache/nifi/jms/processors/PublishJMS.java | 15 ++-- 7 files changed, 90 insertions(+), 22 deletions(-) diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java index adb94fd..6bab920 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-cf-service/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderDefinition.java @@ -35,4 +35,12 @@ public interface JMSConnectionFactoryProviderDefinition extends ControllerServic */ ConnectionFactory getConnectionFactory(); +/** + * Resets {@link ConnectionFactory}. + * Provider should reset {@link ConnectionFactory} only if a copy provided by a client matches + * current {@link ConnectionFactory}. + * @param cachedFactory - {@link ConnectionFactory} cached by client. + */ +void resetConnectionFactory(ConnectionFactory cachedFactory); + } diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java index ecb4e7a..781ce65 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java @@ -139,6 +139,14 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl .build(); } +@Override +public void resetConnectionFactory(ConnectionFactory cachedFactory) { +if (cachedFactory == connectionFactory) { +getLogger().debug("Resetting connection factory"); +connectionFactory = null; +} +} + /** * @return new instance of {@link ConnectionFactory} */ @@ -316,5 +324,4 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl return StandardValidators.NON_EMPTY_VALIDATOR.validate(subject, input, context); } } - } diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java index a293d84..44d8d99 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JndiJmsConnectionFactoryProvider.java @@ -139,6 +139,14 @@ public class JndiJmsConnectionFactoryProvider extends AbstractControllerService } @Override +public synchronized void resetConnectionFactory(ConnectionFactory cachedFactory) { +if (cachedFactory == connectionFactory) { +getLogger().debug("Resetting connection factory"); +connectionFactory = null; +} +} + +@Override public synchronized ConnectionFactory getConnectionFactory() { if (connectionFactory == null) { connectionFactory = lookupConnectionFactory(); diff --git a/nifi-nar-bu
[nifi] branch master updated: NIFI-6000 Catch also IllegalArgumentException in ConvertAvroToORC hive processor. Added support for Avro null types.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e598b30 NIFI-6000 Catch also IllegalArgumentException in ConvertAvroToORC hive processor. Added support for Avro null types. e598b30 is described below commit e598b30d6dc85111762f9618f1b7cfefc68e612b Author: Aleksandr Salatich AuthorDate: Tue Feb 12 18:28:48 2019 +0300 NIFI-6000 Catch also IllegalArgumentException in ConvertAvroToORC hive processor. Added support for Avro null types. Signed-off-by: Matthew Burgess This closes #3302 --- .../apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java | 3 + .../nifi/processors/hive/ConvertAvroToORC.java | 4 +- .../nifi/processors/hive/TestConvertAvroToORC.java | 95 ++ .../org/apache/nifi/util/orc/TestNiFiOrcUtils.java | 50 +++- 4 files changed, 149 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java index 687073e..ce06f82 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/hadoop/hive/ql/io/orc/NiFiOrcUtils.java @@ -244,6 +244,7 @@ public class NiFiOrcUtils { case DOUBLE: case FLOAT: case STRING: +case NULL: return getPrimitiveOrcTypeFromPrimitiveAvroType(fieldType); case UNION: @@ -335,6 +336,7 @@ public class NiFiOrcUtils { case LONG: return TypeInfoFactory.getPrimitiveTypeInfo("bigint"); case BOOLEAN: +case NULL: // ORC has no null type, so just pick the smallest. All values are necessarily null. return TypeInfoFactory.getPrimitiveTypeInfo("boolean"); case BYTES: return TypeInfoFactory.getPrimitiveTypeInfo("binary"); @@ -362,6 +364,7 @@ public class NiFiOrcUtils { case LONG: return "BIGINT"; case BOOLEAN: +case NULL: // Hive has no null type, we picked boolean as the ORC type so use it for Hive DDL too. All values are necessarily null. return "BOOLEAN"; case BYTES: return "BINARY"; diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java index f211ac5..e8ee2a2 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ConvertAvroToORC.java @@ -283,8 +283,8 @@ public class ConvertAvroToORC extends AbstractProcessor { session.transfer(flowFile, REL_SUCCESS); session.getProvenanceReporter().modifyContent(flowFile, "Converted "+totalRecordCount.get()+" records", System.currentTimeMillis() - startTime); -} catch (final ProcessException pe) { -getLogger().error("Failed to convert {} from Avro to ORC due to {}; transferring to failure", new Object[]{flowFile, pe}); +} catch (ProcessException | IllegalArgumentException e) { +getLogger().error("Failed to convert {} from Avro to ORC due to {}; transferring to failure", new Object[]{flowFile, e}); session.transfer(flowFile, REL_FAILURE); } } diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java index 282b42d..f34a647 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/test/java/org/apache/nifi/processors/hive/TestConvertAvroToORC.java @@ -17,10 +17,13 @@ package org.apache.nifi.processors.hive; import org.apache.avro.Schema; +import org.apache.avro.file.DataFileStream; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.g
[nifi] branch master updated: NIFI-4367 Fix on processor for permit deriving script classes from AbstractProcessor or other Records based base classes
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8a197e5 NIFI-4367 Fix on processor for permit deriving script classes from AbstractProcessor or other Records based base classes 8a197e5 is described below commit 8a197e5a96479b4b298aa821a8ab9fdfdc1deb60 Author: Patrice Freydiere AuthorDate: Sat Sep 9 16:43:14 2017 +0200 NIFI-4367 Fix on processor for permit deriving script classes from AbstractProcessor or other Records based base classes Improve UT Add licence and fix import improve filtering properties Signed-off-by: Matthew Burgess This closes #2201 --- .../processors/script/InvokeScriptedProcessor.java | 41 ...FilteredPropertiesValidationContextAdapter.java | 72 ++ .../nifi/script/impl/ValidationContextAdapter.java | 97 ++ .../nifi/processors/script/TestInvokeGroovy.java | 30 ++ .../test_implementingabstractProcessor.groovy | 108 + 5 files changed, 331 insertions(+), 17 deletions(-) diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java index 45439bf..c2df128 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/processors/script/InvokeScriptedProcessor.java @@ -16,6 +16,22 @@ */ package org.apache.nifi.processors.script; +import java.io.File; +import java.io.FileInputStream; +import java.nio.charset.Charset; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.script.Invocable; +import javax.script.ScriptEngine; +import javax.script.ScriptException; + import org.apache.commons.io.IOUtils; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperty; @@ -46,21 +62,7 @@ import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; import org.apache.nifi.script.ScriptingComponentHelper; import org.apache.nifi.script.ScriptingComponentUtils; - -import javax.script.Invocable; -import javax.script.ScriptEngine; -import javax.script.ScriptException; -import java.io.File; -import java.io.FileInputStream; -import java.nio.charset.Charset; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; +import org.apache.nifi.script.impl.FilteredPropertiesValidationContextAdapter; @Tags({"script", "invoke", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"}) @CapabilityDescription("Experimental - Invokes a script engine for a Processor defined in the given script. The script must define " @@ -477,8 +479,13 @@ public class InvokeScriptedProcessor extends AbstractSessionFactoryProcessor { // if there was existing validation errors and the processor loaded successfully if (currentValidationResults.isEmpty() && instance != null) { try { -// defer to the underlying processor for validation -final Collection instanceResults = instance.validate(context); +// defer to the underlying processor for validation, without the +// invokescriptedprocessor properties +final Set innerPropertyDescriptor = new HashSet(scriptingComponentHelper.getDescriptors()); + +ValidationContext innerValidationContext = new FilteredPropertiesValidationContextAdapter(context, innerPropertyDescriptor); +final Collection instanceResults = instance.validate(innerValidationContext); + if (instanceResults != null && instanceResults.size() > 0) { // return the validation results from the underlying instance return instanceResults; diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/impl/FilteredProper
[nifi] branch master updated: NIFI-5974 fix: Fragment Attributes are populated in case no split has occured. Unit test is implemented: testNoSplitterInString
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8e77720 NIFI-5974 fix: Fragment Attributes are populated in case no split has occured. Unit test is implemented: testNoSplitterInString 8e77720 is described below commit 8e777203a02c6e6d9772c9dec86966c9516d56fe Author: a.durov AuthorDate: Fri Jan 25 16:57:13 2019 +0300 NIFI-5974 fix: Fragment Attributes are populated in case no split has occured. Unit test is implemented: testNoSplitterInString NIFI-5974: Fixed Checkstyle violations Signed-off-by: Matthew Burgess This closes #3275 --- .../nifi/processors/standard/SplitContent.java | 48 +++--- .../nifi/processors/standard/TestSplitContent.java | 29 + 2 files changed, 53 insertions(+), 24 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java index e80f76a..4877d6a 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/SplitContent.java @@ -257,36 +257,36 @@ public class SplitContent extends AbstractProcessor { }); long lastOffsetPlusSize = -1L; +final ArrayList splitList = new ArrayList<>(); + if (splits.isEmpty()) { FlowFile clone = session.clone(flowFile); -session.transfer(flowFile, REL_ORIGINAL); -session.transfer(clone, REL_SPLITS); +// finishFragmentAttributes performs .clear() so List must be mutable +splitList.add(clone); logger.info("Found no match for {}; transferring original 'original' and transferring clone {} to 'splits'", new Object[]{flowFile, clone}); -return; -} +} else { +for (final Tuple tuple : splits) { +long offset = tuple.getKey(); +long size = tuple.getValue(); +if (size > 0) { +FlowFile split = session.clone(flowFile, offset, size); +splitList.add(split); +} -final ArrayList splitList = new ArrayList<>(); -for (final Tuple tuple : splits) { -long offset = tuple.getKey(); -long size = tuple.getValue(); -if (size > 0) { -FlowFile split = session.clone(flowFile, offset, size); -splitList.add(split); +lastOffsetPlusSize = offset + size; } -lastOffsetPlusSize = offset + size; -} - -// lastOffsetPlusSize indicates the ending position of the last split. -// if the data didn't end with the byte sequence, we need one final split to run from the end -// of the last split to the end of the content. -long finalSplitOffset = lastOffsetPlusSize; -if (!keepTrailingSequence && !keepLeadingSequence) { -finalSplitOffset += byteSequence.length; -} -if (finalSplitOffset > -1L && finalSplitOffset < flowFile.getSize()) { -FlowFile finalSplit = session.clone(flowFile, finalSplitOffset, flowFile.getSize() - finalSplitOffset); -splitList.add(finalSplit); +// lastOffsetPlusSize indicates the ending position of the last split. +// if the data didn't end with the byte sequence, we need one final split to run from the end +// of the last split to the end of the content. +long finalSplitOffset = lastOffsetPlusSize; +if (!keepTrailingSequence && !keepLeadingSequence) { +finalSplitOffset += byteSequence.length; +} +if (finalSplitOffset > -1L && finalSplitOffset < flowFile.getSize()) { +FlowFile finalSplit = session.clone(flowFile, finalSplitOffset, flowFile.getSize() - finalSplitOffset); +splitList.add(finalSplit); +} } final String fragmentId = finishFragmentAttributes(session, flowFile, splitList); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitContent.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestSplitContent.java index 0dec5a1..dce8f43 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors
[nifi] branch master updated: NIFI-5867 - add thread() EL function to get thread name
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8a3d7a4 NIFI-5867 - add thread() EL function to get thread name 8a3d7a4 is described below commit 8a3d7a453a7792d89f6f099e53cd9fc301efcf4c Author: Pierre Villard AuthorDate: Tue Dec 4 22:28:33 2018 +0100 NIFI-5867 - add thread() EL function to get thread name moved thread name evaluation in evaluate method Signed-off-by: Matthew Burgess This closes #3198 --- .../language/antlr/AttributeExpressionLexer.g | 1 + .../language/antlr/AttributeExpressionParser.g | 2 +- .../language/compile/ExpressionCompiler.java | 4 +++ .../evaluation/functions/ThreadEvaluator.java | 39 ++ .../attribute/expression/language/TestQuery.java | 5 +++ .../main/asciidoc/expression-language-guide.adoc | 18 ++ 6 files changed, 68 insertions(+), 1 deletion(-) diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g index fb3958c..d1b22e9 100644 --- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g +++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g @@ -107,6 +107,7 @@ IP : 'ip'; UUID : 'UUID'; HOSTNAME : 'hostname'; // requires boolean arg: prefer FQDN NOW: 'now'; +THREAD : 'thread'; // 0 arg functions diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g index 4d48ee0..0c63c55 100644 --- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g +++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g @@ -128,7 +128,7 @@ functionCall : functionRef -> ^(FUNCTION_CALL functionRef); booleanLiteral : TRUE | FALSE; -zeroArgStandaloneFunction : (IP | UUID | NOW | NEXT_INT | HOSTNAME | RANDOM) LPAREN! RPAREN!; +zeroArgStandaloneFunction : (IP | UUID | NOW | NEXT_INT | HOSTNAME | THREAD | RANDOM) LPAREN! RPAREN!; oneArgStandaloneFunction : ((TO_LITERAL | MATH | GET_STATE_VALUE)^ LPAREN! anyArg RPAREN!) | (HOSTNAME^ LPAREN! booleanLiteral RPAREN!); standaloneFunction : zeroArgStandaloneFunction | oneArgStandaloneFunction; diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java index 4a4a204..de4333c 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java @@ -91,6 +91,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.functions.Substr import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringBeforeLastEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.SubstringEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.ThreadEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.ToLowerEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.ToRadixEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.ToStringEvaluator; @@ -199,6 +200,7 @@ import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpre import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_AFTER_LAST; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE; import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.SUBSTRING_BEFORE_LAST; +import static org.apache.nifi.attribute.expression.language.antlr.AttributeExpressionParser.THREAD; import sta
[nifi] branch master updated: NIFI-5995 Updated ScriptedLookupService documentation to warn about Jython and removed Jython from the list of supported script engines for it because it's broken now.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new fef41b3 NIFI-5995 Updated ScriptedLookupService documentation to warn about Jython and removed Jython from the list of supported script engines for it because it's broken now. fef41b3 is described below commit fef41b302216934886817fd9335ea4b5d62c4c1e Author: Mike Thomsen AuthorDate: Fri Feb 1 10:53:30 2019 -0500 NIFI-5995 Updated ScriptedLookupService documentation to warn about Jython and removed Jython from the list of supported script engines for it because it's broken now. Signed-off-by: Matthew Burgess This closes #3287 --- .../nifi/lookup/script/ScriptedLookupService.java| 20 ++-- .../apache/nifi/script/ScriptingComponentHelper.java | 6 ++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java index 25ac39a..956a9ce 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/lookup/script/ScriptedLookupService.java @@ -22,6 +22,7 @@ import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnDisabled; import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.ConfigurableComponent; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.RequiredPermission; @@ -53,12 +54,15 @@ import java.util.Map; import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** * A Controller service that allows the user to script the lookup operation to be performed (by LookupRecord, e.g.) */ @Tags({"lookup", "record", "script", "invoke", "groovy", "python", "jython", "jruby", "ruby", "javascript", "js", "lua", "luaj"}) -@CapabilityDescription("Allows the user to provide a scripted LookupService instance in order to enrich records from an incoming flow file.") +@CapabilityDescription("Allows the user to provide a scripted LookupService instance in order to enrich records from " + +"an incoming flow file. Please note, that due to a bug in Jython that remains unresolved, it is not possible to use " + +"Jython to write a script for this service in Python.") @Restricted( restrictions = { @Restriction( @@ -116,7 +120,19 @@ public class ScriptedLookupService extends AbstractScriptedControllerService imp } } List supportedPropertyDescriptors = new ArrayList<>(); - supportedPropertyDescriptors.addAll(scriptingComponentHelper.getDescriptors()); +List _temp = new ArrayList<>(); +_temp.addAll(scriptingComponentHelper.getDescriptors()); +_temp.remove(scriptingComponentHelper.SCRIPT_ENGINE); + +PropertyDescriptor.Builder jythonLessEngineProp = new PropertyDescriptor + .Builder().fromPropertyDescriptor(scriptingComponentHelper.SCRIPT_ENGINE); +List filtered = scriptingComponentHelper.getScriptEngineAllowableValues() +.stream().filter(allowableValue -> !allowableValue.getValue().contains("ython")) +.collect(Collectors.toList()); +jythonLessEngineProp.allowableValues(filtered.toArray(new AllowableValue[filtered.size()])); + +supportedPropertyDescriptors.add(jythonLessEngineProp.build()); +supportedPropertyDescriptors.addAll(_temp); final ConfigurableComponent instance = lookupService.get(); if (instance != null) { diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java index 831d305..dc9255e 100644 --- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptingComponentHelper.java +++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/j
[nifi] branch master updated: Update WriteResourceToStream.java
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new da593a2 Update WriteResourceToStream.java da593a2 is described below commit da593a2e8ebd3e2909c75999e4585991c6a2369c Author: Alexander T <37665177+aleks...@users.noreply.github.com> AuthorDate: Thu Mar 21 08:13:25 2019 +0100 Update WriteResourceToStream.java Thread.currentThread().getContextClassLoader().getResourceAsStream(...) works in UnitTest, but not if the NAR is deployed in /extensions folder. If you want to use the processer the resource 'file.txt' is not found. I changed this to 'getClass().getClassLoader()...' and its working as UnitTest and when deployed Signed-off-by: Matthew Burgess This closes #3381 --- .../main/java/org/apache/nifi/processors/WriteResourceToStream.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java b/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java index 1ce0974..cb5b888 100644 --- a/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java +++ b/nifi-external/nifi-example-bundle/nifi-nifi-example-processors/src/main/java/org/apache/nifi/processors/WriteResourceToStream.java @@ -58,8 +58,8 @@ public class WriteResourceToStream extends AbstractProcessor { relationships.add(REL_SUCCESS); relationships.add(REL_FAILURE); this.relationships = Collections.unmodifiableSet(relationships); -final InputStream resourceStream = Thread.currentThread() -.getContextClassLoader().getResourceAsStream("file.txt"); +final InputStream resourceStream = getClass() +.getClassLoader().getResourceAsStream("file.txt"); try { this.resourceData = IOUtils.toString(resourceStream, Charset.defaultCharset()); } catch (IOException e) {
[nifi] branch master updated: NIFI-6215: Ensure that when we are swapping FlowFiles back into nifi, that we consider forward slashes or back slashes as the same path separator when determining filenam
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 7b94518 NIFI-6215: Ensure that when we are swapping FlowFiles back into nifi, that we consider forward slashes or back slashes as the same path separator when determining filename 7b94518 is described below commit 7b945182af250f3cb820f1d6fdc35bd31d21ba29 Author: Mark Payne AuthorDate: Mon Apr 15 15:24:15 2019 -0400 NIFI-6215: Ensure that when we are swapping FlowFiles back into nifi, that we consider forward slashes or back slashes as the same path separator when determining filename Signed-off-by: Matthew Burgess This closes #3434 --- .../nifi/controller/repository/WriteAheadFlowFileRepository.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java index fdde18c..05dbf88 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/repository/WriteAheadFlowFileRepository.java @@ -420,7 +420,8 @@ public class WriteAheadFlowFileRepository implements FlowFileRepository, SyncLis return null; } -final String withoutTrailing = (swapLocation.endsWith("/") && swapLocation.length() > 1) ? swapLocation.substring(0, swapLocation.length() - 1) : swapLocation; +final String normalizedPath = swapLocation.replace("\\", "/"); +final String withoutTrailing = (normalizedPath.endsWith("/") && normalizedPath.length() > 1) ? normalizedPath.substring(0, normalizedPath.length() - 1) : normalizedPath; final int lastIndex = withoutTrailing.lastIndexOf("/"); if (lastIndex < 0 || lastIndex >= withoutTrailing.length() - 1) { return withoutTrailing;
[nifi] branch master updated: NIFI-6337 Resolved groovy unit test execution problems in nifi-properties-loader and fixed failing unit tests.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new a9d1dd7 NIFI-6337 Resolved groovy unit test execution problems in nifi-properties-loader and fixed failing unit tests. a9d1dd7 is described below commit a9d1dd7b08f7d7a9ab7056393e042dc984e817ba Author: Andy LoPresto AuthorDate: Thu May 30 17:46:12 2019 -0700 NIFI-6337 Resolved groovy unit test execution problems in nifi-properties-loader and fixed failing unit tests. Signed-off-by: Matthew Burgess This closes #3508 --- .../nifi-framework/nifi-properties-loader/pom.xml | 27 - .../AESSensitivePropertyProviderTest.groovy| 10 ++--- .../NiFiPropertiesLoaderGroovyTest.groovy | 44 +++--- .../unreadable_bootstrap/bootstrap.conf| 0 ...nsitive_properties_protected_aes_128.properties | 2 +- 5 files changed, 54 insertions(+), 29 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml index 8b4b970..6b10e61 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/pom.xml @@ -1,4 +1,5 @@ -http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd;> +http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd;> + + +org.codehaus.mojo +build-helper-maven-plugin +1.5 + + +add-test-source +generate-test-sources + +add-test-source + + + +src/test/groovy + + + + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy index 73ae55a..56a702a 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-properties-loader/src/test/groovy/org/apache/nifi/properties/AESSensitivePropertyProviderTest.groovy @@ -17,7 +17,6 @@ package org.apache.nifi.properties import org.bouncycastle.jce.provider.BouncyCastleProvider -import org.bouncycastle.util.encoders.DecoderException import org.bouncycastle.util.encoders.Hex import org.junit.After import org.junit.Assume @@ -302,9 +301,10 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase { SensitivePropertyProvider spp = new AESSensitivePropertyProvider(Hex.decode(getKeyOfSize(keySize))) logger.info("Initialized ${spp.name} with key size ${keySize}") String cipherText = spp.protect(PLAINTEXT) + // Remove the IV from the "complete" cipher text final String MISSING_IV_CIPHER_TEXT = cipherText[18..-1] -logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT.padLeft(163)}") +logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT.padLeft(172)}") def msg = shouldFail(IllegalArgumentException) { spp.unprotect(MISSING_IV_CIPHER_TEXT) @@ -313,9 +313,9 @@ class AESSensitivePropertyProviderTest extends GroovyTestCase { // Remove the IV from the "complete" cipher text but keep the delimiter final String MISSING_IV_CIPHER_TEXT_WITH_DELIMITER = cipherText[16..-1] -logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT_WITH_DELIMITER.padLeft(163)}") +logger.info("Manipulated ${cipherText} to\n${MISSING_IV_CIPHER_TEXT_WITH_DELIMITER.padLeft(172)}") -def msgWithDelimiter = shouldFail(DecoderException) { +def msgWithDelimiter = shouldFail(
[nifi] branch master updated: NIFI-5916 Added an option to enable empty flowfiles to be sent if there are no results from a query.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 407add7 NIFI-5916 Added an option to enable empty flowfiles to be sent if there are no results from a query. 407add7 is described below commit 407add7847f9d79ea9c7b604fd57c3160ff30c06 Author: Mike Thomsen AuthorDate: Sat Feb 16 07:27:25 2019 -0500 NIFI-5916 Added an option to enable empty flowfiles to be sent if there are no results from a query. NIFI-5916 Fixed potential NPE. Signed-off-by: Matthew Burgess This closes #3315 --- .../apache/nifi/processors/mongodb/GetMongo.java | 30 +- .../apache/nifi/processors/mongodb/GetMongoIT.java | 21 +-- 2 files changed, 48 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java index 286c47e..b3b82b0 100644 --- a/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java +++ b/nifi-nar-bundles/nifi-mongodb-bundle/nifi-mongodb-processors/src/main/java/org/apache/nifi/processors/mongodb/GetMongo.java @@ -29,15 +29,18 @@ import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; +import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.Validator; +import org.apache.nifi.context.PropertyContext; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.ProcessContext; import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; +import org.apache.nifi.processor.util.StandardValidators; import org.bson.Document; import org.bson.json.JsonWriterSettings; @@ -58,7 +61,16 @@ import java.util.Set; @WritesAttribute(attribute = GetMongo.COL_NAME, description = "The collection where the results came from.") }) public class GetMongo extends AbstractMongoQueryProcessor { - +public static final PropertyDescriptor SEND_EMPTY_RESULTS = new PropertyDescriptor.Builder() +.name("get-mongo-send-empty") +.displayName("Send Empty Result") +.description("If a query executes successfully, but returns no results, send an empty JSON document " + +"signifying no result.") +.allowableValues("true", "false") +.defaultValue("false") +.addValidator(StandardValidators.BOOLEAN_VALIDATOR) +.required(false) +.build(); static final AllowableValue YES_PP = new AllowableValue("true", "True"); static final AllowableValue NO_PP = new AllowableValue("false", "False"); @@ -94,6 +106,7 @@ public class GetMongo extends AbstractMongoQueryProcessor { _propertyDescriptors.add(DATE_FORMAT); _propertyDescriptors.add(SSL_CONTEXT_SERVICE); _propertyDescriptors.add(CLIENT_AUTH); +_propertyDescriptors.add(SEND_EMPTY_RESULTS); propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors); final Set _relationships = new HashSet<>(); @@ -103,6 +116,12 @@ public class GetMongo extends AbstractMongoQueryProcessor { relationships = Collections.unmodifiableSet(_relationships); } +private boolean sendEmpty; +@OnScheduled +public void onScheduled(PropertyContext context) { +sendEmpty = context.getProperty(SEND_EMPTY_RESULTS).asBoolean(); +} + @Override public Set getRelationships() { return relationships; @@ -191,6 +210,7 @@ public class GetMongo extends AbstractMongoQueryProcessor { it.batchSize(context.getProperty(BATCH_SIZE).evaluateAttributeExpressions(input).asInteger()); } +long sent = 0; try (MongoCursor cursor = it.iterator()) { configureMapper(jsonTypeSetting, dateFormat); @@ -209,6 +229,7 @@ public class GetMongo extends AbstractMongoQueryProcessor { logger.error("Error building batch due to {}", new Object[] {e}); } } +sent++; } if (batc
[nifi] branch master updated: NIFI-5902 Added GeoEnrichIPRecord. NIFI-5902 Updated GeoEnrichIPRecord with more error handling.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3e7816d NIFI-5902 Added GeoEnrichIPRecord. NIFI-5902 Updated GeoEnrichIPRecord with more error handling. 3e7816d is described below commit 3e7816da43cabf95f26f80fcf4dae2dfd848dfa1 Author: Mike Thomsen AuthorDate: Mon Dec 17 15:47:55 2018 -0500 NIFI-5902 Added GeoEnrichIPRecord. NIFI-5902 Updated GeoEnrichIPRecord with more error handling. NIFI-5902 Made changes requested in a review. NIFI-5902 Updated EL support on database path. NIFI-5902 Made updates based on code review. NIFI-5902 Made two changes from a code review. Signed-off-by: Matthew Burgess This closes #3231 --- .../nifi-enrich-bundle/nifi-enrich-nar/pom.xml | 6 + .../nifi-enrich-processors/pom.xml | 45 +++ .../apache/nifi/processors/AbstractEnrichIP.java | 3 +- .../apache/nifi/processors/GeoEnrichIPRecord.java | 352 + .../services/org.apache.nifi.processor.Processor | 1 + .../apache/nifi/processors/GeoEnrichTestUtils.java | 97 ++ .../apache/nifi/processors/TestGeoEnrichIP.java| 76 + .../nifi/processors/TestGeoEnrichIPRecord.java | 167 ++ .../src/test/resources/avro/record_schema.avsc | 42 +++ .../src/test/resources/json/one_record.json| 5 + .../src/test/resources/json/one_record_no_geo.json | 4 + .../test/resources/json/two_records_for_split.json | 8 + 12 files changed, 731 insertions(+), 75 deletions(-) diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml index 864cf95..96f377c 100644 --- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml +++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-nar/pom.xml @@ -31,6 +31,12 @@ org.apache.nifi +nifi-standard-services-api-nar +${project.version} +nar + + +org.apache.nifi nifi-enrich-processors diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml index 68e0061..213757e 100644 --- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml +++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/pom.xml @@ -88,5 +88,50 @@ 1.6.5 test + +org.apache.nifi +nifi-record-serialization-service-api +compile + + +org.apache.nifi +nifi-record-path +1.10.0-SNAPSHOT +compile + + +org.apache.nifi +nifi-record-serialization-services +1.10.0-SNAPSHOT +test + + +org.apache.nifi +nifi-mock-record-utils +1.10.0-SNAPSHOT +test + + +org.apache.nifi +nifi-schema-registry-service-api +test + + + + + +org.apache.rat +apache-rat-plugin + + + src/test/resources/avro/record_schema.avsc + src/test/resources/json/one_record.json + src/test/resources/json/one_record_no_geo.json + src/test/resources/json/two_records_for_split.json + + + + + diff --git a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java index 30226a9..e726976 100644 --- a/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java +++ b/nifi-nar-bundles/nifi-enrich-bundle/nifi-enrich-processors/src/main/java/org/apache/nifi/processors/AbstractEnrichIP.java @@ -50,6 +50,7 @@ public abstract class AbstractEnrichIP extends AbstractProcessor { .description("Path to Maxmind IP Enrichment Database File") .required(true) .addValidator(StandardValidators.FILE_EXISTS_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .build(); public static final PropertyDescriptor IP_ADDRESS_ATTRIBUTE = new PropertyDescriptor.Builder() @@ -88,7 +89,7 @@ public abstract class AbstractEnrichIP extends AbstractProcessor { @OnScheduled public void onScheduled(final ProcessContext conte
[nifi] branch master updated: NIFI-6228 Fixed order of operations bug that prevented SSLContext from being handled correctly.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new c352c2c NIFI-6228 Fixed order of operations bug that prevented SSLContext from being handled correctly. c352c2c is described below commit c352c2cc3efc03821aa4ef16b6e7ffa45c99843f Author: Mike Thomsen AuthorDate: Thu Jun 13 18:05:30 2019 -0400 NIFI-6228 Fixed order of operations bug that prevented SSLContext from being handled correctly. NIFI-6228 Added unit test for SSL configuration. Signed-off-by: Matthew Burgess This closes #3535 --- .../elasticsearch/ElasticSearchClientService.java | 2 ++ .../nifi-elasticsearch-client-service/pom.xml | 5 .../ElasticSearchClientServiceImpl.java| 2 +- .../ElasticSearch5ClientService_IT.groovy | 26 + .../src/test/resources/keystore.jks| Bin 0 -> 3077 bytes .../src/test/resources/truststore.jks | Bin 0 -> 911 bytes 6 files changed, 34 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java index 188c7bb..57f359d 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service-api/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientService.java @@ -19,6 +19,7 @@ package org.apache.nifi.elasticsearch; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.components.PropertyDescriptor; +import org.apache.nifi.components.Validator; import org.apache.nifi.controller.ControllerService; import org.apache.nifi.expression.ExpressionLanguageScope; import org.apache.nifi.processor.util.StandardValidators; @@ -46,6 +47,7 @@ public interface ElasticSearchClientService extends ControllerService { + "connections. This service only applies if the Elasticsearch endpoint(s) have been secured with TLS/SSL.") .required(false) .identifiesControllerService(SSLContextService.class) +.addValidator(Validator.VALID) .build(); PropertyDescriptor USERNAME = new PropertyDescriptor.Builder() .name("el-cs-username") diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml index dfdb75a..a1bbdb4 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/pom.xml @@ -155,6 +155,11 @@ 1.10.0-SNAPSHOT compile + +org.apache.nifi +nifi-ssl-context-service +test + diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java index b240189..b37ba0c 100644 --- a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java +++ b/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/main/java/org/apache/nifi/elasticsearch/ElasticSearchClientServiceImpl.java @@ -128,7 +128,7 @@ public class ElasticSearchClientServiceImpl extends AbstractControllerService im final SSLContext sslContext; try { -sslContext = (sslService != null && sslService.isKeyStoreConfigured() || sslService.isTrustStoreConfigured()) +sslContext = (sslService != null && (sslService.isKeyStoreConfigured() || sslService.isTrustStoreConfigured())) ? sslService.createSSLContext(SSLContextService.ClientAuth.NONE) : null; } catch (Exception e) { getLogger().error("Error building up SSL Context from the supplied configuration.", e); diff --git a/nifi-nar-bundles/nifi-elasticsearch-bundle/nifi-elasticsearch-client-service/src/test/groovy/org/apache/nifi/elasticsearch/integration/ElasticSearch5ClientService_IT.groovy
[nifi] 01/02: NIFI-5537 Create Neo4J cypher execution processor
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git commit 1b28c7b45c3eb4b7c58214ae64688226fcb866b1 Author: mans2singh AuthorDate: Sun Aug 19 19:18:13 2018 -0700 NIFI-5537 Create Neo4J cypher execution processor NIFI-5537 Renamed bundle based on review comments NIFI-5537 Updated neo4j setup and version documentation and default values based on review comments NIFI-5537 - Changed exception, logging, and json parser based on review comments NIFI-5537 - Check debug enabled before logging at debug level NIFI-5537 - Updated property displayname/description and corrected test file names NIFI-5537 - Added note about dummy password required in case authentication is disabled on server NIFI-5537 - Use SSLContext --- nifi-assembly/NOTICE | 5 + nifi-assembly/pom.xml | 6 + .../nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml | 45 +++ .../src/main/resources/META-INF/LICENSE| 202 ++ .../src/main/resources/META-INF/NOTICE | 31 +++ .../nifi-neo4j-processors/pom.xml | 91 +++ .../neo4j/AbstractNeo4JCypherExecutor.java | 301 + .../nifi/processors/neo4j/Neo4JCypherExecutor.java | 207 ++ .../services/org.apache.nifi.processor.Processor | 15 + .../processors/neo4j/ITNeo4JCypherExecutor.java| 219 +++ .../processors/neo4j/TestNeo4JCypherExecutor.java | 272 +++ nifi-nar-bundles/nifi-neo4j-bundle/pom.xml | 43 +++ nifi-nar-bundles/pom.xml | 1 + 13 files changed, 1438 insertions(+) diff --git a/nifi-assembly/NOTICE b/nifi-assembly/NOTICE index 9850ad8..8c1c0d6 100644 --- a/nifi-assembly/NOTICE +++ b/nifi-assembly/NOTICE @@ -1144,6 +1144,11 @@ The following binary components are provided under the Apache Software License v Copyright 2010-2012 RethinkDB +(ASLv2) Neo4j Java Driver + Neo4j + Copyright © 2002-2018 Neo4j Sweden AB (referred to in this notice as "Neo4j") + [http://neo4j.com] + (ASLv2) Carrotsearch HPPC The following NOTICE information applies: HPPC borrowed code, ideas or both from: diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 5087dd6..abec31c 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -399,6 +399,12 @@ language governing permissions and limitations under the License. --> org.apache.nifi +nifi-neo4j-nar +1.8.0-SNAPSHOT +nar + + +org.apache.nifi nifi-avro-nar 1.10.0-SNAPSHOT nar diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml new file mode 100644 index 000..16ee10f --- /dev/null +++ b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/pom.xml @@ -0,0 +1,45 @@ + + +http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd;> +4.0.0 + + +org.apache.nifi +nifi-neo4j-bundle +1.8.0-SNAPSHOT + + +nifi-neo4j-nar +nar + +true +true + + + + +org.apache.nifi +nifi-standard-services-api-nar +1.8.0-SNAPSHOT +nar + + +org.apache.nifi +nifi-neo4j-processors + + + + diff --git a/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE new file mode 100644 index 000..d645695 --- /dev/null +++ b/nifi-nar-bundles/nifi-neo4j-bundle/nifi-neo4j-nar/src/main/resources/META-INF/LICENSE @@ -0,0 +1,202 @@ + + Apache License + Version 2.0, January 2004 +http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition, + "control" means (i) the power, direct or indirect, t
[nifi] branch master updated (c352c2c -> 82f177c)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from c352c2c NIFI-6228 Fixed order of operations bug that prevented SSLContext from being handled correctly. new 1b28c7b NIFI-5537 Create Neo4J cypher execution processor new 82f177c NIFI-5537 Refactored to be named Cypher Bundle instead. NIFI-5537 Added skeletons of services, apis, etc. NIFI-5537 Renamed classes to reflect that they are for cypher and not neo4j specific NIFI-5537 Added start of client API for doing streaming execution. NIFI-5537 Completed refactor of unit and integration tests. NIFI-5537 Added OpenCypherClientService. NIFI-5537 Updated to gremlin-core 3.3.5 to get better SSL configuration options. NIFI-5537 Added SSL support to Open [...] The 2 revisions listed above as "new" are entirely new to this repository and will be described in separate emails. The revisions listed as "add" were already present in the repository and have only been added to this reference. Summary of changes: nifi-assembly/NOTICE | 5 + nifi-assembly/pom.xml | 24 ++ .../nifi-graph-client-service-api-nar/pom.xml | 42 ++ .../nifi-graph-client-service-api/pom.xml | 45 ++ .../org/apache/nifi/graph/GraphClientService.java | 35 ++ .../nifi/graph/GraphQueryResultCallback.java | 24 ++ .../apache/nifi/graph/TinkerPopClientService.java | 25 ++ .../nifi-graph-bundle/nifi-graph-nar/pom.xml | 45 ++ .../src/main/resources/META-INF/LICENSE| 0 .../src/main/resources/META-INF/NOTICE | 44 ++ .../nifi-graph-processors/pom.xml | 91 + .../processors/graph/AbstractGraphExecutor.java| 98 + .../nifi/processors/graph/ExecuteGraphQuery.java | 185 + .../services/org.apache.nifi.processor.Processor | 15 + .../additionalDetails.html | 37 ++ .../processors/graph/MockCypherClientService.java | 56 +++ .../processors/graph/TestExecuteGraphQuery.java| 101 + .../nifi-neo4j-cypher-service-nar/pom.xml | 37 ++ .../src/main/resources/META-INF/LICENSE| 265 .../src/main/resources/META-INF/NOTICE | 152 +++ .../nifi-neo4j-cypher-service/pom.xml | 104 + .../nifi/graph/Neo4JCypherClientService.java | 317 +++ .../org.apache.nifi.controller.ControllerService | 16 + .../apache/nifi/graph/ITNeo4JCypherExecutor.java | 152 +++ .../java/org/apache/nifi/graph/MockProcessor.java | 47 +++ .../nifi-other-graph-services-nar/pom.xml | 37 ++ .../src/main/resources/META-INF/LICENSE| 254 .../src/main/resources/META-INF/NOTICE | 452 + .../nifi-other-graph-services/pom.xml | 112 + .../nifi/graph/AbstractTinkerpopClientService.java | 109 + .../apache/nifi/graph/GremlinClientService.java| 93 + .../apache/nifi/graph/OpenCypherClientService.java | 103 + .../org.apache.nifi.controller.ControllerService | 17 + .../additionalDetails.html | 54 +++ .../apache/nifi/graph/GremlinClientServiceIT.java | 75 .../java/org/apache/nifi/graph/MockProcessor.java | 52 +++ .../nifi/graph/OpenCypherClientServiceIT.java | 96 + .../nifi/graph/TestableGremlinClientService.java | 26 ++ .../src/test/resources/setup.gremlin | 41 ++ .../src/test/resources/teardown.gremlin| 19 + nifi-nar-bundles/nifi-graph-bundle/pom.xml | 49 +++ nifi-nar-bundles/pom.xml | 1 + 42 files changed, 3552 insertions(+) create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api-nar/pom.xml create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/pom.xml create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphClientService.java create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/GraphQueryResultCallback.java create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-client-service-api/src/main/java/org/apache/nifi/graph/TinkerPopClientService.java create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/pom.xml copy nifi-nar-bundles/{nifi-windows-event-log-bundle/nifi-windows-event-log-nar => nifi-graph-bundle/nifi-graph-nar}/src/main/resources/META-INF/LICENSE (100%) create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-nar/src/main/resources/META-INF/NOTICE create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/pom.xml create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/ma
[nifi] branch master updated: NIFI-6257: removing duplicate table row from administration guide
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9194726 NIFI-6257: removing duplicate table row from administration guide 9194726 is described below commit 9194726b35ca06fc590c06674bb0cf4c7200d7b4 Author: Endre Zoltan Kovacs AuthorDate: Fri May 3 09:55:30 2019 +0200 NIFI-6257: removing duplicate table row from administration guide Signed-off-by: Matthew Burgess This closes #3461 --- nifi-docs/src/main/asciidoc/administration-guide.adoc | 2 -- 1 file changed, 2 deletions(-) diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index adcf58a..e0a20ff 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -3108,8 +3108,6 @@ from the remote node before considering the communication with the node a failur |`nifi.cluster.node.max.concurrent.requests`|The maximum number of outstanding web requests that can be replicated to nodes in the cluster. If this number of requests is exceeded, the embedded Jetty server will return a "409: Conflict" response. This property defaults to `100`. |`nifi.cluster.firewall.file`|The location of the node firewall file. This is a file that may be used to list all the nodes that are allowed to connect to the cluster. It provides an additional layer of security. This value is blank by default, meaning that no firewall file is to be used. -|`nifi.cluster.flow.election.max.wait.time`|Specifies the amount of time to wait before electing a Flow as the "correct" Flow. If the number of Nodes that have voted is equal to the number specified by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast. -|`nifi.cluster.flow.election.max.candidates`|Specifies the number of Nodes required in the cluster to cause early election of Flows. This allows the Nodes in the cluster to avoid having to wait a long time before starting processing if we reach at least this number of nodes in the cluster. |`nifi.cluster.flow.election.max.wait.time`|Specifies the amount of time to wait before electing a Flow as the "correct" Flow. If the number of Nodes that have voted is equal to the number specified by the `nifi.cluster.flow.election.max.candidates` property, the cluster will not wait this long. The default value is `5 mins`. Note that the time starts as soon as the first vote is cast. |`nifi.cluster.flow.election.max.candidates`|Specifies the number of Nodes required in the cluster to cause early election of Flows. This allows the Nodes in the cluster to avoid having to wait a
[nifi] branch master updated: NIFI-6413 Remove nifi-ssl-context-service dependency from nifi-prometheus-nar
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 0857d5f NIFI-6413 Remove nifi-ssl-context-service dependency from nifi-prometheus-nar 0857d5f is described below commit 0857d5f89dd47863698e54544dd5871975a154b4 Author: Koji Kawamura AuthorDate: Tue Jul 2 15:02:28 2019 +0900 NIFI-6413 Remove nifi-ssl-context-service dependency from nifi-prometheus-nar Signed-off-by: Matthew Burgess This closes #3565 --- .../nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml| 5 - 1 file changed, 5 deletions(-) diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml index 8290222..90f141c 100644 --- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml +++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml @@ -65,11 +65,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/ma test -org.apache.nifi -nifi-ssl-context-service -1.10.0-SNAPSHOT - - org.apache.nifi nifi-security-utils 1.10.0-SNAPSHOT
[nifi] branch analytics-framework updated: NIFI-6566 - Refactor to decouple model instance from status analytics object. Also allow configurable model from nifi.properties
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch analytics-framework in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/analytics-framework by this push: new 7d6e1b5 NIFI-6566 - Refactor to decouple model instance from status analytics object. Also allow configurable model from nifi.properties 7d6e1b5 is described below commit 7d6e1b509550fcaede785f1fa5c329bc85329920 Author: Yolanda Davis AuthorDate: Tue Aug 20 19:59:58 2019 -0400 NIFI-6566 - Refactor to decouple model instance from status analytics object. Also allow configurable model from nifi.properties NIFI-6566 - changes to allow scoring configurations for model in nifi.properties NIFI-6566 - added default implementation value to NiFiProperties NIFI-6566 - correction to default variable name in NiFiProperties, removed unnecessary init method from ConnectionStatusAnalytics Signed-off-by: Matthew Burgess This closes #3663 --- .../status/analytics/StatusAnalyticsModel.java | 4 + .../java/org/apache/nifi/util/NiFiProperties.java | 7 +- .../resources/NiFiProperties/conf/nifi.properties | 3 +- .../src/main/asciidoc/administration-guide.adoc| 4 + .../org/apache/nifi/controller/FlowController.java | 27 ++- .../CachingConnectionStatusAnalyticsEngine.java| 27 ++- .../analytics/ConnectionStatusAnalytics.java | 185 + .../analytics/ConnectionStatusAnalyticsEngine.java | 26 ++- .../analytics/StatusAnalyticsModelMapFactory.java | 106 ...Model.java => StatusMetricExtractFunction.java} | 20 +-- .../models/BivariateStatusAnalyticsModel.java | 17 +- .../models/MultivariateStatusAnalyticsModel.java | 50 -- ...tSquaresMSAM.java => OrdinaryLeastSquares.java} | 29 ++-- ...leRegressionBSAM.java => SimpleRegression.java} | 32 ++-- ...ontroller.status.analytics.StatusAnalyticsModel | 16 ++ ...TestCachingConnectionStatusAnalyticsEngine.java | 19 ++- .../analytics/TestConnectionStatusAnalytics.java | 25 ++- .../TestConnectionStatusAnalyticsEngine.java | 8 +- .../analytics/TestStatusAnalyticsEngine.java | 29 +++- .../TestStatusAnalyticsModelMapFactory.java| 66 ...uresMSAM.java => TestOrdinaryLeastSqaures.java} | 37 ++--- ...gressionBSAM.java => TestSimpleRegression.java} | 10 +- .../src/test/resources/conf/nifi.properties| 5 +- .../nar/StandardExtensionDiscoveringManager.java | 34 ++-- .../nifi-framework/nifi-resources/pom.xml | 3 + .../src/main/resources/conf/nifi.properties| 5 +- 26 files changed, 497 insertions(+), 297 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java index 4869a0f..8d601e6 100644 --- a/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java +++ b/nifi-api/src/main/java/org/apache/nifi/controller/status/analytics/StatusAnalyticsModel.java @@ -16,12 +16,16 @@ */ package org.apache.nifi.controller.status.analytics; +import java.util.Map; import java.util.stream.Stream; public interface StatusAnalyticsModel { void learn(Stream features, Stream labels); Double predict(Double[] feature); +Double predictVariable(Integer predictVariableIndex, Map knownVariablesWithIndex, Double label); Boolean supportsOnlineLearning(); +Map getScores(); +void clear(); } diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index aa00793..32e273e 100644 --- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -240,6 +240,9 @@ public abstract class NiFiProperties { // analytics properties public static final String ANALYTICS_PREDICTION_INTERVAL = "nifi.analytics.predict.interval"; +public static final String ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION = "nifi.analytics.connection.model.implementation"; +public static final String ANALYTICS_CONNECTION_MODEL_SCORE_NAME= "nifi.analytics.connection.model.score.name"; +public static final String ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD = "nifi.analytics.connection.model.score.threshold"; // defaults public static final Boolean DEFAULT_AUTO_RESUME_STATE = true; @@ -313,7 +316,9 @@ public abstract class NiFiProperties { // analytics defaults public static final String DEFAULT_ANALYTICS_PREDICTION_INTERVAL = "3 mins"; - +public final
[nifi] branch master updated: NIFI-6424 Created a proper transit URL for Gremlin and OpenCypher services.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new fc3477b NIFI-6424 Created a proper transit URL for Gremlin and OpenCypher services. fc3477b is described below commit fc3477bd69be066ba7f75b90f9e58e18ee3b176c Author: Mike Thomsen AuthorDate: Wed Jul 3 08:06:28 2019 -0400 NIFI-6424 Created a proper transit URL for Gremlin and OpenCypher services. Signed-off-by: Matthew Burgess This closes #3571 --- .../java/org/apache/nifi/graph/AbstractTinkerpopClientService.java | 7 +++ .../src/main/java/org/apache/nifi/graph/GremlinClientService.java | 2 +- .../main/java/org/apache/nifi/graph/OpenCypherClientService.java | 2 -- .../test/java/org/apache/nifi/graph/GremlinClientServiceIT.java| 2 ++ .../test/java/org/apache/nifi/graph/OpenCypherClientServiceIT.java | 3 +++ 5 files changed, 13 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java index 4bf7bec..929e6df 100644 --- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java +++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/AbstractTinkerpopClientService.java @@ -85,11 +85,15 @@ public abstract class AbstractTinkerpopClientService extends AbstractControllerS .keyStoreType(service.getKeyStoreType()) .trustStore(service.getTrustStoreFile()) .trustStorePassword(service.getTrustStorePassword()); +usesSSL = true; } return builder; } +boolean usesSSL; +protected String transitUrl; + protected Cluster buildCluster(ConfigurationContext context) { String contactProp = context.getProperty(CONTACT_POINTS).evaluateAttributeExpressions().getValue(); int port = context.getProperty(PORT).evaluateAttributeExpressions().asInteger(); @@ -104,6 +108,9 @@ public abstract class AbstractTinkerpopClientService extends AbstractControllerS builder = setupSSL(context, builder); +transitUrl = String.format("gremlin%s://%s:%s%s", usesSSL ? "+ssl" : "", +contactProp, port, path); + return builder.create(); } } diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java index 4b51623..a79094f 100644 --- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java +++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java @@ -88,6 +88,6 @@ public class GremlinClientService extends AbstractTinkerpopClientService impleme @Override public String getTransitUrl() { -return null; +return transitUrl; } } diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java index 4fb4699..47e44ee 100644 --- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java +++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/OpenCypherClientService.java @@ -40,14 +40,12 @@ import java.util.Map; @Tags({ "cypher", "opencypher", "graph", "database", "janus" }) public class OpenCypherClientService extends AbstractTinkerpopClientService implements GraphClientService { private volatile Driver gremlinDriver; -private volatile String transitUrl; @OnEnabled public void onEnabled(ConfigurationContext context) { Cluster cluster = buildCluster(context); gremlinDriver = GremlinDatabase.driver(cluster); -transitUrl = String.format("gremlin://%s", context.getProperty(CONTACT_POINTS).getValue()); } @OnDisabled diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/GremlinClientServiceIT.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/test/java/org/apache/nifi/graph/GremlinClientServiceIT.jav
[nifi] branch master updated: NIFI-6425 Made executeQuery able to reconnect to the Gremlin cluster.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 768a7b8 NIFI-6425 Made executeQuery able to reconnect to the Gremlin cluster. 768a7b8 is described below commit 768a7b8c004f2ab0a4284ab29479cd917f486c1b Author: Mike Thomsen AuthorDate: Thu Jul 4 07:50:21 2019 -0400 NIFI-6425 Made executeQuery able to reconnect to the Gremlin cluster. Signed-off-by: Matthew Burgess This closes #3572 --- .../org/apache/nifi/graph/GremlinClientService.java| 18 -- 1 file changed, 16 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java index a79094f..e001fd0 100644 --- a/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java +++ b/nifi-nar-bundles/nifi-graph-bundle/nifi-other-graph-services/src/main/java/org/apache/nifi/graph/GremlinClientService.java @@ -37,9 +37,11 @@ public class GremlinClientService extends AbstractTinkerpopClientService impleme private Cluster cluster; protected Client client; public static final String NOT_SUPPORTED = "NOT_SUPPORTED"; +private ConfigurationContext context; @OnEnabled public void onEnabled(ConfigurationContext context) { +this.context = context; cluster = buildCluster(context); client = cluster.connect(); } @@ -52,8 +54,7 @@ public class GremlinClientService extends AbstractTinkerpopClientService impleme cluster = null; } -@Override -public Map executeQuery(String query, Map parameters, GraphQueryResultCallback handler) { +public Map doQuery(String query, Map parameters, GraphQueryResultCallback handler) { try { Iterator iterator = client.submit(query, parameters).iterator(); long count = 0; @@ -87,6 +88,19 @@ public class GremlinClientService extends AbstractTinkerpopClientService impleme } @Override +public Map executeQuery(String query, Map parameters, GraphQueryResultCallback handler) { +try { +return doQuery(query, parameters, handler); +} catch (Exception ex) { +cluster.close(); +client.close(); +cluster = buildCluster(context); +client = cluster.connect(); +return doQuery(query, parameters, handler); +} +} + +@Override public String getTransitUrl() { return transitUrl; }
[nifi] branch master updated: - Removed unused AUTOCREATE_PARTITIONS from PutHive3Streaming - Renamed PARTITION_VALUES to STATIC_PARTITION_VALUES for correctness and better understanding - STATIC_PART
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9071e5b - Removed unused AUTOCREATE_PARTITIONS from PutHive3Streaming - Renamed PARTITION_VALUES to STATIC_PARTITION_VALUES for correctness and better understanding - STATIC_PARTITION_VALUES descriptions clearly states that having that property filler implies Hive Static Partitioning 9071e5b is described below commit 9071e5baa7a8af03392fce703d539e9fa94980be Author: Alessandro D'Armiento AuthorDate: Sat Aug 3 13:49:44 2019 +0200 - Removed unused AUTOCREATE_PARTITIONS from PutHive3Streaming - Renamed PARTITION_VALUES to STATIC_PARTITION_VALUES for correctness and better understanding - STATIC_PARTITION_VALUES descriptions clearly states that having that property filler implies Hive Static Partitioning NIFI-6536: Additional documentation for Static Partition Values Signed-off-by: Matthew Burgess This closes #3631 --- .../nifi/processors/hive/PutHive3Streaming.java| 35 -- .../org/apache/nifi/util/hive/HiveOptions.java | 6 2 files changed, 12 insertions(+), 29 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java index 2224b06..5558c79 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java @@ -84,9 +84,10 @@ import java.util.stream.Collectors; import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES; @Tags({"hive", "streaming", "put", "database", "store"}) -@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. " -+ "The partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in " -+ "each record should be field A.") +@CapabilityDescription("This processor uses Hive Streaming to send flow file records to an Apache Hive 3.0+ table. If 'Static Partition Values' is not set, then " ++ "the partition values are expected to be the 'last' fields of each record, so if the table is partitioned on column A for example, then the last field in " ++ "each record should be field A. If 'Static Partition Values' is set, those values will be used as the partition values, and any record fields corresponding to " ++ "partition columns will be ignored.") @WritesAttributes({ @WritesAttribute(attribute = "hivestreaming.record.count", description = "This attribute is written on the flow files routed to the 'success' " + "and 'failure' relationships, and contains the number of records from the incoming flow file. All records in a flow file are committed as a single transaction."), @@ -151,28 +152,19 @@ public class PutHive3Streaming extends AbstractProcessor { .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .build(); -static final PropertyDescriptor PARTITION_VALUES = new PropertyDescriptor.Builder() +static final PropertyDescriptor STATIC_PARTITION_VALUES = new PropertyDescriptor.Builder() .name("hive3-stream-part-vals") -.displayName("Partition Values") +.displayName("Static Partition Values") .description("Specifies a comma-separated list of the values for the partition columns of the target table. If the incoming records all have the same values " + "for the partition columns, those values can be entered here, resulting in a performance gain. If specified, this property will often contain " + "Expression Language, for example if PartitionRecord is upstream and two partitions 'name' and 'age' are used, then this property can be set to " -+ "${name},${age}.") ++ "${name},${age}. If this property is set, the values will be used as the partition values, and any record fields corresponding to " ++ "partition columns will be ignored. If this property is not set, then the partition values are expected to be the last fields of each record.") .required(false
[nifi] branch analytics-framework updated (7d6e1b5 -> e9986c2)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch analytics-framework in repository https://gitbox.apache.org/repos/asf/nifi.git. from 7d6e1b5 NIFI-6566 - Refactor to decouple model instance from status analytics object. Also allow configurable model from nifi.properties add e9986c2 NIFI-6585 - Refactored tests to use mocked models and extract functions. Added check in ConnectionStatusAnalytics to confirm expected model by type No new revisions were added by this update. Summary of changes: .../analytics/ConnectionStatusAnalytics.java | 16 +- .../analytics/TestConnectionStatusAnalytics.java | 370 - .../analytics/TestStatusAnalyticsEngine.java | 34 +- 3 files changed, 324 insertions(+), 96 deletions(-)
[nifi] branch master updated: NIFI-6621: Add support for Druid schema-less dimensions
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new a672294 NIFI-6621: Add support for Druid schema-less dimensions a672294 is described below commit a672294f3f3edf2f3fbd05a20d0585dc1a431c79 Author: samhjelmfelt AuthorDate: Wed Sep 4 16:20:27 2019 -0500 NIFI-6621: Add support for Druid schema-less dimensions NIFI-6621: Small change to fix failing tests NIFI-6621: Minor style changes Signed-off-by: Matthew Burgess This closes #3693 --- .../controller/druid/DruidTranquilityController.java | 18 +++--- .../druid/DruidTranquilityControllerTest.java | 2 -- .../druid/MockDruidTranquilityController.java | 3 ++- 3 files changed, 13 insertions(+), 10 deletions(-) diff --git a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java index 78aab4c..4d69699 100644 --- a/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java +++ b/nifi-nar-bundles/nifi-druid-bundle/nifi-druid-controller-service/src/main/java/org/apache/nifi/controller/druid/DruidTranquilityController.java @@ -231,8 +231,7 @@ public class DruidTranquilityController extends AbstractControllerService implem public static final PropertyDescriptor DIMENSIONS_LIST = new PropertyDescriptor.Builder() .name("druid-cs-dimensions-list") .displayName("Dimension Fields") -.description("A comma separated list of field names that will be stored as dimensions on ingest.") -.required(true) +.description("A comma separated list of field names that will be stored as dimensions on ingest. Set to empty string for schema-less dimensions.") .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .build(); @@ -400,7 +399,7 @@ public class DruidTranquilityController extends AbstractControllerService implem transitUri = String.format(FIREHOSE_PATTERN, dataSource) + ";indexServicePath=" + indexService; -final List dimensions = getDimensions(dimensionsStringList); +final DruidDimensions dimensions = getDimensions(dimensionsStringList); final List aggregator = getAggregatorList(aggregatorJSON); final Timestamper> timestamper = new Timestamper>() { @@ -446,14 +445,14 @@ public class DruidTranquilityController extends AbstractControllerService implem } Beam> buildBeam(String dataSource, String indexService, String discoveryPath, int clusterPartitions, int clusterReplication, -String segmentGranularity, String queryGranularity, String windowPeriod, String firehoseGracePeriod, String indexRetryPeriod, List dimensions, +String segmentGranularity, String queryGranularity, String windowPeriod, String firehoseGracePeriod, String indexRetryPeriod, DruidDimensions dimensions, List aggregator, Timestamper> timestamper, TimestampSpec timestampSpec) { return DruidBeams.builder(timestamper) .curator(curator) .discoveryPath(discoveryPath) .location(DruidLocation.create(DruidEnvironment.create(indexService, FIREHOSE_PATTERN), dataSource)) .timestampSpec(timestampSpec) - .rollup(DruidRollup.create(DruidDimensions.specific(dimensions), aggregator, QueryGranularity.fromString(queryGranularity))) +.rollup(DruidRollup.create(dimensions, aggregator, QueryGranularity.fromString(queryGranularity))) .tuning( ClusteredBeamTuning .builder() @@ -518,7 +517,7 @@ public class DruidTranquilityController extends AbstractControllerService implem } } -private List getDimensions(String dimensionStringList) { +private DruidDimensions getDimensions(String dimensionStringList) { List dimensionList = new ArrayList<>(); if (dimensionStringList != null) { Arrays.stream(dimensionStringList.split(",")) @@ -526,7 +525,12 @@ public class DruidTranquilityController extends AbstractControllerService implem .map(String::trim) .forEach(dimensionList::add); } -return dimensionList; +i
[nifi] branch analytics-framework updated: NIFI-6510 - Fix issue displaying estimated time to back pressure in connection summary table when only one of the predictions is known.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch analytics-framework in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/analytics-framework by this push: new 60d9ce4 NIFI-6510 - Fix issue displaying estimated time to back pressure in connection summary table when only one of the predictions is known. 60d9ce4 is described below commit 60d9ce427c2b1af9b291d09e19b47de4edeba753 Author: Rob Fellows AuthorDate: Fri Sep 6 13:03:45 2019 -0400 NIFI-6510 - Fix issue displaying estimated time to back pressure in connection summary table when only one of the predictions is known. Signed-off-by: Matthew Burgess This closes #3705 --- .../src/main/webapp/js/nf/summary/nf-summary-table.js | 15 --- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js index b665bbd..3585356 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/summary/nf-summary-table.js @@ -103,17 +103,26 @@ { label: 'size', percent: percentUseBytes } ]; -var minPrediction = _.minBy(predictions, 'timeToBackPressure'); +// if one of the queues is already at backpressure, return now as the prediction var maxActual = _.maxBy(actualQueuePercents, 'percent'); - if (maxActual.percent >= 100) { // currently experiencing back pressure return 'now (' + maxActual.label + ')'; -} else if (minPrediction.timeToBackPressure < 0) { +} + +// filter out the predictions that are unknown +var knownPredictions = predictions.filter(function(p) { +return p.timeToBackPressure >= 0; +}); + +if (_.isEmpty(knownPredictions)) { // there is not a valid time-to-back-pressure prediction available return 'NA'; } +// there is at least one valid prediction, return the minimum time to back pressure +var minPrediction = _.minBy(knownPredictions, 'timeToBackPressure'); + var formatted = nfCommon.formatPredictedDuration(minPrediction.timeToBackPressure); return nfCommon.escapeHtml(formatted) + ' (' + minPrediction.label + ')'; };
[nifi] branch master updated: fix deserialization issues with NiFiRecordSerDe for hive3streaming
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 3d1bb09 fix deserialization issues with NiFiRecordSerDe for hive3streaming 3d1bb09 is described below commit 3d1bb09ff85462b08ffadcd2cbc8a5d6036a7cdc Author: korir AuthorDate: Fri May 31 19:50:09 2019 +0300 fix deserialization issues with NiFiRecordSerDe for hive3streaming NIFI-6295: Refactored NiFiRecordSerDe to handle nested complex types NIFI-6295: Incorporated review comments NIFI-6295: Refactored unit tests for NiFiRecordSerDe Signed-off-by: Matthew Burgess This closes #3684 --- .../nifi-hive-bundle/nifi-hive3-processors/pom.xml | 33 +- .../org/apache/hive/streaming/NiFiRecordSerDe.java | 248 +++--- .../apache/hive/streaming/TestNiFiRecordSerDe.java | 373 + .../processors/hive/TestPutHive3Streaming.java | 95 +- .../src/test/resources/nested-map-input.json | 18 + .../src/test/resources/nested-map-schema.avsc | 48 +++ 6 files changed, 679 insertions(+), 136 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml index e790283..38f72a5 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml @@ -13,7 +13,7 @@ See the License for the specific language governing permissions and limitations under the License. --> -http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd;> +http://maven.apache.org/POM/4.0.0; xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd;> 4.0.0 @@ -140,5 +140,36 @@ hamcrest-all test + +org.apache.nifi +nifi-schema-registry-service-api +1.10.0-SNAPSHOT +test + + +org.apache.nifi +nifi-record-serialization-services +1.10.0-SNAPSHOT +test + + +org.apache.avro +avro + + + + + + +org.apache.rat +apache-rat-plugin + + + src/test/resources/nested-map-input.json + + + + + diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java index e628474..51a06c8 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/hive/streaming/NiFiRecordSerDe.java @@ -27,6 +27,8 @@ import org.apache.hadoop.hive.serde2.SerDeUtils; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.StandardStructObjectInspector; +import org.apache.hadoop.hive.serde2.typeinfo.ListTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.MapTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.StructTypeInfo; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; @@ -38,14 +40,17 @@ import org.apache.hive.common.util.HiveStringUtils; import org.apache.hive.common.util.TimestampParser; import org.apache.nifi.avro.AvroTypeUtil; import org.apache.nifi.logging.ComponentLog; -import org.apache.nifi.serialization.MalformedRecordException; import org.apache.nifi.serialization.RecordReader; +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.type.ArrayDataType; +import org.apache.nifi.serialization.record.type.MapDataType; import org.apache.nifi.serialization.record.util.DataTypeUtils; -import java.io.IOException; +import java.math.BigDecimal; import java.sql.Timestamp; import java.util.ArrayList; import java.util.Arrays; @@ -57,6
[nifi] branch master updated: NIFI-6649 - added separate query interval configuration for observation queries NIFI-6649 - documentation update
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 8e1452a NIFI-6649 - added separate query interval configuration for observation queries NIFI-6649 - documentation update 8e1452a is described below commit 8e1452a3f342ee45dfd589d343c9ecf6a7cf6825 Author: Yolanda M. Davis AuthorDate: Tue Sep 10 15:54:42 2019 -0400 NIFI-6649 - added separate query interval configuration for observation queries NIFI-6649 - documentation update NIFI-6649 - add debug logging for score and prediction information NIFI-6649 - fix to ensure counts return minimum value of 0 if not infinite or NaN Signed-off-by: Matthew Burgess This closes #3719 --- .../java/org/apache/nifi/util/NiFiProperties.java | 2 ++ .../src/main/asciidoc/administration-guide.adoc| 3 ++- .../org/apache/nifi/controller/FlowController.java | 13 ++- .../CachingConnectionStatusAnalyticsEngine.java| 5 ++-- .../analytics/ConnectionStatusAnalytics.java | 27 -- .../analytics/ConnectionStatusAnalyticsEngine.java | 6 - ...TestCachingConnectionStatusAnalyticsEngine.java | 8 --- .../TestConnectionStatusAnalyticsEngine.java | 5 ++-- .../analytics/TestStatusAnalyticsEngine.java | 5 ++-- .../nifi-framework/nifi-resources/pom.xml | 1 + .../src/main/resources/conf/nifi.properties| 1 + 11 files changed, 62 insertions(+), 14 deletions(-) diff --git a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java index 7da514c..afcd268 100644 --- a/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java +++ b/nifi-commons/nifi-properties/src/main/java/org/apache/nifi/util/NiFiProperties.java @@ -241,6 +241,7 @@ public abstract class NiFiProperties { // analytics properties public static final String ANALYTICS_PREDICTION_ENABLED = "nifi.analytics.predict.enabled"; public static final String ANALYTICS_PREDICTION_INTERVAL = "nifi.analytics.predict.interval"; +public static final String ANALYTICS_QUERY_INTERVAL = "nifi.analytics.query.interval"; public static final String ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION = "nifi.analytics.connection.model.implementation"; public static final String ANALYTICS_CONNECTION_MODEL_SCORE_NAME= "nifi.analytics.connection.model.score.name"; public static final String ANALYTICS_CONNECTION_MODEL_SCORE_THRESHOLD = "nifi.analytics.connection.model.score.threshold"; @@ -318,6 +319,7 @@ public abstract class NiFiProperties { // analytics defaults public static final String DEFAULT_ANALYTICS_PREDICTION_ENABLED = "false"; public static final String DEFAULT_ANALYTICS_PREDICTION_INTERVAL = "3 mins"; +public static final String DEFAULT_ANALYTICS_QUERY_INTERVAL = "3 mins"; public final static String DEFAULT_ANALYTICS_CONNECTION_MODEL_IMPLEMENTATION = "org.apache.nifi.controller.status.analytics.models.OrdinaryLeastSquares"; public static final String DEFAULT_ANALYTICS_CONNECTION_SCORE_NAME = "rSquared"; public static final double DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD = .90; diff --git a/nifi-docs/src/main/asciidoc/administration-guide.adoc b/nifi-docs/src/main/asciidoc/administration-guide.adoc index 432ca9e..8c5cac9 100644 --- a/nifi-docs/src/main/asciidoc/administration-guide.adoc +++ b/nifi-docs/src/main/asciidoc/administration-guide.adoc @@ -2389,7 +2389,7 @@ In order to generate predictions, local status snapshot history is queried to ob NiFi evaluates the model's effectiveness before sending prediction information by using the model's R-Squared score by default. One important note: R-Square is a measure of how close the regression line fits the observation data vs. how accurate the prediction will be; therefore there may be some measure of error. If the R-Squared score for the calculated model meets the configured threshold (as defined by `nifi.analytics.connection.model.score.threshold`) then the model will be used for [...] -The prediction interval `nifi.analytics.predict.interval` can be configured to project out further when back pressure will occur. Predictions further out in time require more observations stored locally to generate an effective model. This may also require tuning of the model's scoring threshold value to select a score which can offer reasonable predictions. +The prediction interval `nifi.analytics.predict.interval` can be configured to project out further when back pressure will occur. The prediction query interval `nifi.analytics
[nifi] branch master updated: NIFI-6649 - repaired test due to min zero fix
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 5106b76 NIFI-6649 - repaired test due to min zero fix 5106b76 is described below commit 5106b764da021149054231d08c25381c736964bb Author: Yolanda M. Davis AuthorDate: Thu Sep 12 15:49:34 2019 -0400 NIFI-6649 - repaired test due to min zero fix NIFI-6649 - added check if logging debug is enabled Signed-off-by: Matthew Burgess This closes #3730 --- .../analytics/ConnectionStatusAnalytics.java | 2 +- .../analytics/TestConnectionStatusAnalytics.java | 24 +++--- 2 files changed, 13 insertions(+), 13 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java index 7abba4b..e152339 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java @@ -97,7 +97,7 @@ public class ConnectionStatusAnalytics implements StatusAnalytics { LOG.debug("Refreshing model with new data for connection id: {} ", connectionIdentifier); model.learn(Stream.of(features), Stream.of(values)); -if(MapUtils.isNotEmpty(model.getScores())){ +if(LOG.isDebugEnabled() && MapUtils.isNotEmpty(model.getScores())){ model.getScores().forEach((key, value) -> { LOG.debug("Model Scores for prediction metric {} for connection id {}: {}={} ", metric, connectionIdentifier, key, value); }); diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java index 5d9279f..ffa86a8 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/status/analytics/TestConnectionStatusAnalytics.java @@ -287,12 +287,12 @@ public class TestConnectionStatusAnalytics { } @Test -public void testCannotPredictNextIntervalBytesNegative() { +public void testNextIntervalBytesZero() { Map> modelMap = getModelMap("queuedBytes",.9,-1.0,-1.0); ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap); Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalBytes(); assertNotNull(nextIntervalBytes); -assert (nextIntervalBytes == -1); +assert (nextIntervalBytes == 0); } @Test @@ -335,30 +335,30 @@ public class TestConnectionStatusAnalytics { } @Test -public void testCannotPredictNextIntervalCountNegative() { +public void testGetNextIntervalCountZero() { Map> modelMap = getModelMap("queuedCount",.9,-1.0,-1.0); ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap); -Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount(); -assertNotNull(nextIntervalBytes); -assert (nextIntervalBytes == -1); +Long nextIntervalCount = connectionStatusAnalytics.getNextIntervalCount(); +assertNotNull(nextIntervalCount); +assert (nextIntervalCount == 0); } @Test public void testCannotPredictNextIntervalCountNaN() { Map> modelMap = getModelMap("queuedCount",.9,Double.NaN,Double.NaN); ConnectionStatusAnalytics connectionStatusAnalytics = getConnectionStatusAnalytics(modelMap); -Long nextIntervalBytes = connectionStatusAnalytics.getNextIntervalCount(); -assertNotNull(nextIntervalBytes); -assert (nextIntervalBytes == -1); +Long nextIntervalCount = connectionStatusAnalytics.getNextIntervalCount(); +assertNotNull(nextIntervalCount); +assert (nextIntervalCount == -1); } @Test public void testCannotPredi
[nifi] branch master updated: NIFI-6642 - JsonPath support for adding array element
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9ec6314 NIFI-6642 - JsonPath support for adding array element 9ec6314 is described below commit 9ec6314687dfc78ff28566770777e5ec0544f238 Author: mans2singh AuthorDate: Sat Sep 7 11:11:35 2019 -0400 NIFI-6642 - JsonPath support for adding array element NIFI-6642 - Updated json file used in tests Clarified function doc Signed-off-by: Matthew Burgess This closes #3711 --- .../language/antlr/AttributeExpressionLexer.g | 1 + .../language/antlr/AttributeExpressionParser.g | 2 +- .../language/compile/ExpressionCompiler.java | 46 +++-- .../evaluation/functions/JsonPathAddEvaluator.java | 37 ++ .../evaluation/functions/JsonPathSetEvaluator.java | 29 ++- ...Evaluator.java => JsonPathUpdateEvaluator.java} | 21 ++-- .../attribute/expression/language/TestQuery.java | 34 + .../src/test/resources/json/address-book.json | 3 +- .../main/asciidoc/expression-language-guide.adoc | 58 +- 9 files changed, 185 insertions(+), 46 deletions(-) diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g index 0f93016..44fa387 100644 --- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g +++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionLexer.g @@ -189,6 +189,7 @@ REPLACE_FIRST : 'replaceFirst'; REPLACE_ALL : 'replaceAll'; IF_ELSE : 'ifElse'; JSON_PATH_SET : 'jsonPathSet'; +JSON_PATH_ADD : 'jsonPathAdd'; PAD_LEFT : 'padLeft'; PAD_RIGHT : 'padRight'; diff --git a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g index 30576b7..6a62caa 100644 --- a/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g +++ b/nifi-commons/nifi-expression-language/src/main/antlr3/org/apache/nifi/attribute/expression/language/antlr/AttributeExpressionParser.g @@ -78,7 +78,7 @@ zeroArgString : (TO_UPPER | TO_LOWER | TRIM | TO_STRING | URL_ENCODE | URL_DECOD oneArgString : ((SUBSTRING_BEFORE | SUBSTRING_BEFORE_LAST | SUBSTRING_AFTER | SUBSTRING_AFTER_LAST | REPLACE_NULL | REPLACE_EMPTY | PREPEND | APPEND | STARTS_WITH | ENDS_WITH | CONTAINS | JOIN | JSON_PATH | JSON_PATH_DELETE | FROM_RADIX) LPAREN! anyArg RPAREN!) | (TO_RADIX LPAREN! anyArg (COMMA! anyArg)? RPAREN!); -twoArgString : ((REPLACE | REPLACE_FIRST | REPLACE_ALL | IF_ELSE | JSON_PATH_SET) LPAREN! anyArg COMMA! anyArg RPAREN!) | +twoArgString : ((REPLACE | REPLACE_FIRST | REPLACE_ALL | IF_ELSE | JSON_PATH_SET | JSON_PATH_ADD) LPAREN! anyArg COMMA! anyArg RPAREN!) | ((SUBSTRING | FORMAT | PAD_LEFT | PAD_RIGHT) LPAREN! anyArg (COMMA! anyArg)? RPAREN!); fiveArgString : GET_DELIMITED_FIELD LPAREN! anyArg (COMMA! anyArg (COMMA! anyArg (COMMA! anyArg (COMMA! anyArg)?)?)?)? RPAREN!; diff --git a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java index 20b10d0..8edb832 100644 --- a/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java +++ b/nifi-commons/nifi-expression-language/src/main/java/org/apache/nifi/attribute/expression/language/compile/ExpressionCompiler.java @@ -64,6 +64,7 @@ import org.apache.nifi.attribute.expression.language.evaluation.functions.InEval import org.apache.nifi.attribute.expression.language.evaluation.functions.IndexOfEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.IsEmptyEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.IsNullEvaluator; +import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathAddEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathDeleteEvaluator; import org.apache.nifi.attribute.expression.language.evaluation.functions.JsonPathEvalua
[nifi] branch master updated: NIFI-6534 Improve logging in PutHive3Streaming processor Added logging for PutHive3Streaming when routing to failure
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 2baafcc NIFI-6534 Improve logging in PutHive3Streaming processor Added logging for PutHive3Streaming when routing to failure 2baafcc is described below commit 2baafcc2e688d84f1531aa95bf7a6c6203fbdb6b Author: Tamas Palfy AuthorDate: Fri Aug 9 10:38:24 2019 +0200 NIFI-6534 Improve logging in PutHive3Streaming processor Added logging for PutHive3Streaming when routing to failure Signed-off-by: Matthew Burgess This closes #3640 --- .../nifi-hive-bundle/nifi-hive3-processors/pom.xml | 5 + .../nifi/processors/hive/PutHive3Streaming.java| 15 ++ .../processors/hive/TestPutHive3Streaming.java | 24 ++ 3 files changed, 44 insertions(+) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml index 33a148d..e790283 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/pom.xml @@ -135,5 +135,10 @@ 1.10.0-SNAPSHOT test + +org.hamcrest +hamcrest-all +test + diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java index 5f3445a..2224b06 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java @@ -431,6 +431,11 @@ public class PutHive3Streaming extends AbstractProcessor { if (rollbackOnFailure) { throw new ProcessException(rrfe); } else { +log.error( +"Failed to create {} for {} - routing to failure", +new Object[]{RecordReader.class.getSimpleName(), flowFile}, +rrfe +); session.transfer(flowFile, REL_FAILURE); } } @@ -445,6 +450,11 @@ public class PutHive3Streaming extends AbstractProcessor { updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten())); updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName()); flowFile = session.putAllAttributes(flowFile, updateAttributes); +log.error( +"Exception while processing {} - routing to failure", +new Object[]{flowFile}, +e +); session.transfer(flowFile, REL_FAILURE); } } catch (DiscontinuedException e) { @@ -479,6 +489,11 @@ public class PutHive3Streaming extends AbstractProcessor { updateAttributes.put(HIVE_STREAMING_RECORD_COUNT_ATTR, Long.toString(hiveStreamingConnection.getConnectionStats().getRecordsWritten())); updateAttributes.put(ATTR_OUTPUT_TABLES, options.getQualifiedTableName()); flowFile = session.putAllAttributes(flowFile, updateAttributes); +log.error( +"Exception while trying to stream {} to hive - routing to failure", +new Object[]{flowFile}, +se +); session.transfer(flowFile, REL_FAILURE); } diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java index 6acabe3..c82f55c 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/test/java/org/apache/nifi/processors/hive/TestPutHive3Streaming.java @@ -97,10 +97,14 @@ import java.util.function.BiFunction; import static org.apache.nifi.processors.hive.AbstractHive3QLProcessor.ATTR_OUTPUT_TABLES; import static org.apache.nifi.processors.hive.PutHive3Streaming.HIVE_STREAMING_RECORD_COUNT_ATTR; import static org.apache.nifi.processors.hive.PutHive3Streaming.KERBEROS_CREDENTIALS_SERVICE; +import static org.hamcre
[nifi] branch master updated: NIFI-6854 - Added option to ignore condition errors in rules. Test correction
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new a37b57e NIFI-6854 - Added option to ignore condition errors in rules. Test correction a37b57e is described below commit a37b57e96b7e0d1d048f0d4344b1cec692216617 Author: Yolanda M. Davis AuthorDate: Thu Nov 7 17:21:48 2019 -0500 NIFI-6854 - Added option to ignore condition errors in rules. Test correction NIFI-6854 - change warning to debug Signed-off-by: Matthew Burgess This closes #3876 --- .../nifi-easyrules-service/pom.xml | 1 + .../org/apache/nifi/rules/RulesMVELCondition.java | 54 .../org/apache/nifi/rules/RulesSPELCondition.java | 75 +++ .../nifi/rules/engine/EasyRulesEngineService.java | 21 ++- .../org/apache/nifi/rules/TestRulesFactory.java| 2 +- .../rules/engine/TestEasyRulesEngineService.java | 148 - ...st_spel_rules.json => test_bad_spel_rules.json} | 2 +- .../src/test/resources/test_mvel_rules.json| 2 +- .../src/test/resources/test_mvel_rules.yml | 2 +- .../src/test/resources/test_nifi_rules.json| 2 +- .../src/test/resources/test_nifi_rules.yml | 2 +- .../src/test/resources/test_spel_rules.json| 2 +- .../src/test/resources/test_spel_rules.yml | 2 +- 13 files changed, 297 insertions(+), 18 deletions(-) diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml index 7e98f34..801cbb3 100644 --- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml @@ -79,6 +79,7 @@ src/test/resources/test_mvel_rules.json src/test/resources/test_mvel_rules.yml src/test/resources/test_spel_rules.json + src/test/resources/test_bad_spel_rules.json src/test/resources/test_spel_rules.yml diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java new file mode 100644 index 000..7bd615f --- /dev/null +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java @@ -0,0 +1,54 @@ +/* + * 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.rules; + +import org.jeasy.rules.api.Condition; +import org.jeasy.rules.api.Facts; +import org.jeasy.rules.mvel.MVELCondition; +import org.mvel2.MVEL; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.Serializable; + +public class RulesMVELCondition implements Condition { + +private static final Logger LOGGER = LoggerFactory.getLogger(MVELCondition.class); +private String expression; +private Serializable compiledExpression; +private boolean ignoreConditionErrors; + +public RulesMVELCondition(String expression, boolean ignoreConditionErrors) { +this.expression = expression; +this.compiledExpression = MVEL.compileExpression(expression); +this.ignoreConditionErrors = ignoreConditionErrors; +} + +public boolean evaluate(Facts facts) { +try { +return (Boolean)MVEL.executeExpression(this.compiledExpression, facts.asMap()); +} catch (Exception ex) { +if(ignoreConditionErrors) { +LOGGER.debug("Unable to evaluate expression: '" + this.expression + "' on facts: " + facts, ex); +return false; +} else{ +throw ex; +} +} +} +} diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi
[nifi] branch master updated: NIFI-6775 Added fix to support Oracle Float type conversion to Avro
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 7ac4157 NIFI-6775 Added fix to support Oracle Float type conversion to Avro 7ac4157 is described below commit 7ac41576e3711ccc752d730f0b39a474235c701c Author: Atul Mahajan AuthorDate: Mon Oct 14 19:13:22 2019 +0530 NIFI-6775 Added fix to support Oracle Float type conversion to Avro Signed-off-by: Matthew Burgess This closes #3812 --- .../src/main/java/org/apache/nifi/util/db/JdbcCommon.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java index 9d57f10..d437930 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java @@ -569,7 +569,9 @@ public class JdbcCommon { if (meta.getPrecision(i) > 0) { // When database returns a certain precision, we can rely on that. decimalPrecision = meta.getPrecision(i); -decimalScale = meta.getScale(i); +//For the float data type Oracle return decimalScale < 0 which cause is not expected to org.apache.avro.LogicalTypes +//Hence falling back to default scale if decimalScale < 0 +decimalScale = meta.getScale(i) > 0 ? meta.getScale(i) : options.defaultScale; } else { // If not, use default precision. decimalPrecision = options.defaultPrecision;
[nifi] branch master updated: NIFI-6771 : Added fix to support Oracle Binary float and Binary double conversion to Avro type
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e3f72c5 NIFI-6771 : Added fix to support Oracle Binary float and Binary double conversion to Avro type e3f72c5 is described below commit e3f72c5ed09382144c01caa5ff7cfc9c72a9ce38 Author: Atul Mahajan AuthorDate: Tue Oct 15 16:45:10 2019 +0530 NIFI-6771 : Added fix to support Oracle Binary float and Binary double conversion to Avro type Fixed CheckStyle errors Signed-off-by: Matthew Burgess This closes #3815 --- .../src/main/java/org/apache/nifi/util/db/JdbcCommon.java | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java index d437930..34c27c2 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-database-utils/src/main/java/org/apache/nifi/util/db/JdbcCommon.java @@ -345,7 +345,10 @@ public class JdbcCommon { byte[] bytes = rs.getBytes(i); ByteBuffer bb = ByteBuffer.wrap(bytes); rec.put(i - 1, bb); - +} else if (javaSqlType == 100) { // Handle Oracle BINARY_FLOAT data type +rec.put(i - 1, rs.getFloat(i)); +} else if (javaSqlType == 101) { // Handle Oracle BINARY_DOUBLE data type +rec.put(i - 1, rs.getDouble(i)); } else if (value instanceof Byte) { // tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT // But value is returned by JDBC as java.lang.Byte @@ -553,10 +556,12 @@ public class JdbcCommon { case FLOAT: case REAL: +case 100: //Oracle BINARY_FLOAT type builder.name(columnName).type().unionOf().nullBuilder().endNull().and().floatType().endUnion().noDefault(); break; case DOUBLE: +case 101: //Oracle BINARY_DOUBLE type builder.name(columnName).type().unionOf().nullBuilder().endNull().and().doubleType().endUnion().noDefault(); break;
[nifi] branch master updated: NIFI-6842 - Introduce MetricsEventReportingTask
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new c998a72 NIFI-6842 - Introduce MetricsEventReportingTask c998a72 is described below commit c998a7259aca73e716433d0582b30d7ed986c4b2 Author: Yolanda M. Davis AuthorDate: Tue Nov 5 14:25:58 2019 -0500 NIFI-6842 - Introduce MetricsEventReportingTask NIFI-6842 - Added AlertHandler for bulletin reporting. Update ReportingTask meta data. NIFI-6842 - corrected display names in action handlers, included metrics option for alert handlers, small refactor in reporting task NIFI-6842 - updated docs and tags NIFI-6842 - Added documentation for handlers. Signed-off-by: Matthew Burgess This closes #3874 --- nifi-assembly/pom.xml | 12 + .../apache/nifi/rules/handlers/AlertHandler.java | 169 + .../nifi/rules/handlers/ExpressionHandler.java | 10 +- .../org/apache/nifi/rules/handlers/LogHandler.java | 29 ++- .../nifi/rules/handlers/RecordSinkHandler.java | 5 +- .../org.apache.nifi.controller.ControllerService | 3 +- .../additionalDetails.html | 39 +++ .../additionalDetails.html | 38 +++ .../additionalDetails.html | 38 +++ .../additionalDetails.html | 37 +++ .../nifi/rules/handlers/TestAlertHandler.java | 264 + .../nifi-sql-reporting-tasks/pom.xml | 6 + .../reporting/sql/MetricsEventReportingTask.java | 105 .../nifi/reporting/sql/QueryNiFiReportingTask.java | 84 +-- .../nifi/reporting/sql/util/QueryMetricsUtil.java | 114 + .../org.apache.nifi.reporting.ReportingTask| 3 +- .../additionalDetails.html | 34 +++ ...ask.java => TestMetricsEventReportingTask.java} | 176 +- .../reporting/sql/TestQueryNiFiReportingTask.java | 21 +- .../rules/MockPropertyContextActionHandler.java| 76 ++ .../nifi/rules/engine/MockRulesEngineService.java | 47 21 files changed, 1094 insertions(+), 216 deletions(-) diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index f3381a7..b036e65 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -878,6 +878,18 @@ language governing permissions and limitations under the License. --> 1.11.0-SNAPSHOT nar + +org.apache.nifi +nifi-easyrules-nar +1.11.0-SNAPSHOT +nar + + +org.apache.nifi +nifi-rules-action-handler-nar +1.11.0-SNAPSHOT +nar + diff --git a/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AlertHandler.java b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AlertHandler.java new file mode 100644 index 000..234ea3c --- /dev/null +++ b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AlertHandler.java @@ -0,0 +1,169 @@ +/* + * 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.rules.handlers; + +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.PropertyDescriptor; +import org.apache.nifi.context.PropertyContext; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.controller.ControllerServiceInitializationContext; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.util.S
[nifi] branch master updated: NIFI-6896 Standardize hadoop.version for everything that uses nifi-hadoop-libraries
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 5cfc68d NIFI-6896 Standardize hadoop.version for everything that uses nifi-hadoop-libraries 5cfc68d is described below commit 5cfc68d26d97ff57c44ea1ea4cc610740a1b6079 Author: Bryan Bende AuthorDate: Tue Nov 19 13:27:59 2019 -0500 NIFI-6896 Standardize hadoop.version for everything that uses nifi-hadoop-libraries NIFI-6896 Adding hadoop-aws, hadoop-azure, and hadoop-azure-datalake to nifi-hadoop-libraries-nar and nifi-ranger-nar so other filesystem impls will be available NIFI-6896 Introduce profiles to optionally include additional hadoop modules to reduce default build size NIFI-6896 Updating licenses and notices to account for new optional dependencies NIFI-6896 Add profile for hadoop-cloud-storage Signed-off-by: Matthew Burgess This closes #3900 --- nifi-assembly/LICENSE | 20 +- .../nifi-extension-utils/nifi-hadoop-utils/pom.xml | 4 +- .../processors/hadoop/AbstractHadoopProcessor.java | 2 +- .../nifi-hadoop-record-utils/pom.xml | 4 +- .../nifi-flume-processors/pom.xml | 1 - .../nifi-hdfs-processors/pom.xml | 4 +- .../nifi-hadoop-libraries-nar/pom.xml | 74 +++--- .../src/main/resources/META-INF/LICENSE| 66 --- .../src/main/resources/META-INF/NOTICE | 14 .../nifi-parquet-processors/pom.xml| 4 +- .../src/main/resources/META-INF/LICENSE| 20 +- .../src/main/resources/META-INF/NOTICE | 14 .../nifi-ranger-bundle/nifi-ranger-plugin/pom.xml | 66 +++ pom.xml| 3 + 14 files changed, 248 insertions(+), 48 deletions(-) diff --git a/nifi-assembly/LICENSE b/nifi-assembly/LICENSE index a99d16d..d61cb40 100644 --- a/nifi-assembly/LICENSE +++ b/nifi-assembly/LICENSE @@ -2884,4 +2884,22 @@ The binary distribution of this product bundles the 'ASM' library which is avail INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF - THE POSSIBILITY OF SUCH DAMAGE. \ No newline at end of file + THE POSSIBILITY OF SUCH DAMAGE. + +The binary distribution of this product bundles 'Azure SDK for Java' which is available under an MIT license. + + Copyright (c) 2015 Microsoft Azure + + Permission is hereby granted, free of charge, to any person obtaining a copy of this software and + associated documentation files (the Software), to deal in the Software without restriction, including + without limitation the rights to use, copy, modify, merge, publish, distribute, sublicense, and/or + sell copies of the Software, and to permit persons to whom the Software is furnished to do so, subject + to the following conditions: + + The above copyright notice and this permission notice shall be included in all copies or substantial portions of the Software. + + THE SOFTWARE IS PROVIDED *AS IS*, WITHOUT WARRANTY OF ANY KIND, EXPRESS OR IMPLIED, INCLUDING BUT NOT + LIMITED TO THE WARRANTIES OF MERCHANTABILITY, FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN + NO EVENT SHALL THE AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER LIABILITY, + WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, OUT OF OR IN CONNECTION WITH THE + SOFTWARE OR THE USE OR OTHER DEALINGS IN THE SOFTWARE. \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml index c9d6c73..5bfee37 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/pom.xml @@ -23,9 +23,7 @@ nifi-hadoop-utils 1.11.0-SNAPSHOT jar - -3.0.0 - + org.apache.nifi diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java index c46fd63..e742d13 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-hadoop-utils/src/main/java/org/apache/nifi/processors/hadoop/AbstractHadoopProcessor.java
[nifi] branch master updated: NIFI-6890 Support configuring rules in controller service configuration
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 5813048 NIFI-6890 Support configuring rules in controller service configuration 5813048 is described below commit 58130485a3ae3387eefef85b6115563d9a9aeb6c Author: Yolanda M. Davis AuthorDate: Thu Nov 21 17:25:28 2019 -0500 NIFI-6890 Support configuring rules in controller service configuration Signed-off-by: Matthew Burgess This closes #3902 --- .../java/org/apache/nifi/rules/RulesFactory.java | 49 --- .../nifi/rules/engine/EasyRulesEngineService.java | 55 +++-- .../additionalDetails.html | 6 +- .../org/apache/nifi/rules/TestRulesFactory.java| 68 ++--- .../rules/engine/TestEasyRulesEngineService.java | 69 ++ 5 files changed, 211 insertions(+), 36 deletions(-) diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesFactory.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesFactory.java index 18b6b1c..177231d 100644 --- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesFactory.java +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesFactory.java @@ -25,10 +25,9 @@ import org.jeasy.rules.support.YamlRuleDefinitionReader; import org.yaml.snakeyaml.Yaml; import org.yaml.snakeyaml.constructor.Constructor; -import java.io.File; +import java.io.ByteArrayInputStream; import java.io.FileInputStream; import java.io.FileNotFoundException; -import java.io.FileReader; import java.io.InputStream; import java.io.InputStreamReader; import java.util.ArrayList; @@ -51,25 +50,36 @@ public class RulesFactory { NIFI, MVEL, SPEL; } -public static List createRules(String ruleFile, String ruleFileType, String rulesFileFormat) throws Exception{ +public static List createRulesFromFile(String ruleFile, String ruleFileType, String rulesFileFormat) throws Exception { +InputStream rulesInputStream = new FileInputStream(ruleFile); +return createRules(rulesInputStream, ruleFileType, rulesFileFormat); +} + +public static List createRulesFromString(String rulesBody, String ruleFileType, String rulesFileFormat) throws Exception { +InputStream rulesInputStream = new ByteArrayInputStream(rulesBody.getBytes()); +return createRules(rulesInputStream, ruleFileType, rulesFileFormat); +} + +private static List createRules(InputStream rulesInputStream, String ruleFileType, String rulesFileFormat) throws Exception { FileFormat fileFormat = FileFormat.valueOf(rulesFileFormat); -switch (fileFormat){ +switch (fileFormat) { case NIFI: -return createRulesFromNiFiFormat(ruleFile, ruleFileType); +return createRulesFromNiFiFormat(rulesInputStream, ruleFileType); case MVEL: case SPEL: -return createRulesFromEasyRulesFormat(ruleFile, ruleFileType, rulesFileFormat); +return createRulesFromEasyRulesFormat(rulesInputStream, ruleFileType, rulesFileFormat); default: return null; } + } -private static List createRulesFromEasyRulesFormat(String ruleFile, String ruleFileType, String ruleFileFormat) throws Exception{ +private static List createRulesFromEasyRulesFormat(InputStream rulesInputStream, String ruleFileType, String ruleFileFormat) throws Exception { RuleDefinitionReader reader = FileType.valueOf(ruleFileType).equals(FileType.YAML) - ? new YamlRuleDefinitionReader() : new JsonRuleDefinitionReader(); +? new YamlRuleDefinitionReader() : new JsonRuleDefinitionReader(); -List ruleDefinitions = reader.read(new FileReader(ruleFile)); +List ruleDefinitions = reader.read(new InputStreamReader(rulesInputStream)); return ruleDefinitions.stream().map(ruleDefinition -> { @@ -81,7 +91,7 @@ public class RulesFactory { List actions = ruleDefinition.getActions().stream().map(ruleAction -> { Action action = new Action(); action.setType("EXPRESSION"); -Map attributes = new HashMap<>(); +Map attributes = new HashMap<>(); attributes.put("command", ruleAction); attributes.put("type", ruleFileFormat); action.setAttributes(attributes); @@ -93,23 +103,21 @@ public class RulesFactory { }).collect(Collectors.toList()); } -
[nifi] branch master updated: NIFI-6885: - Fixed bug when calling logUnbuildableDestination with "jms_destination" instead of actual destination name value
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new ae2b3d6 NIFI-6885: - Fixed bug when calling logUnbuildableDestination with "jms_destination" instead of actual destination name value ae2b3d6 is described below commit ae2b3d6b45fafb83bedd86f4d6da8d74a4b6ee6d Author: Joe Ferner AuthorDate: Tue Nov 19 10:29:46 2019 -0500 NIFI-6885: - Fixed bug when calling logUnbuildableDestination with "jms_destination" instead of actual destination name value Signed-off-by: Matthew Burgess This closes #3895 --- .../src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java| 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java index 506de49..1ea61b6 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/processors/JMSPublisher.java @@ -105,14 +105,14 @@ final class JMSPublisher extends JMSWorker { if (destination != null) { message.setJMSReplyTo(destination); } else { -logUnbuildableDestination(entry.getKey(), JmsHeaders.REPLY_TO); +logUnbuildableDestination(entry.getValue(), JmsHeaders.REPLY_TO); } } else if (entry.getKey().equals(JmsHeaders.DESTINATION)) { Destination destination = buildDestination(entry.getValue()); if (destination != null) { message.setJMSDestination(destination); } else { -logUnbuildableDestination(entry.getKey(), JmsHeaders.DESTINATION); +logUnbuildableDestination(entry.getValue(), JmsHeaders.DESTINATION); } } else { // not a special attribute handled above, so send it as a property using the specified property type
[nifi] branch master updated: NIFI-6917 - fix doc for dynamic properties variables support in JMS Connection Factory Provider
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 41fef55 NIFI-6917 - fix doc for dynamic properties variables support in JMS Connection Factory Provider 41fef55 is described below commit 41fef551c3cc4e33e9586351f4507b4f090be3a6 Author: Pierre Villard AuthorDate: Wed Dec 4 03:45:04 2019 +0100 NIFI-6917 - fix doc for dynamic properties variables support in JMS Connection Factory Provider Signed-off-by: Matthew Burgess This closes #3914 --- .../nifi/jms/cf/JMSConnectionFactoryProvider.java | 4 +++- .../jms/cf/JMSConnectionFactoryProviderTest.java| 21 - 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java index b831162..b8382a1 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/main/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProvider.java @@ -63,7 +63,8 @@ import org.slf4j.LoggerFactory; @DynamicProperty(name = "The name of a Connection Factory configuration property.", value = "The value of a given Connection Factory configuration property.", description = "The properties that are set following Java Beans convention where a property name is derived from the 'set*' method of the vendor " + "specific ConnectionFactory's implementation. For example, 'com.ibm.mq.jms.MQConnectionFactory.setChannel(String)' would imply 'channel' " -+ "property and 'com.ibm.mq.jms.MQConnectionFactory.setTransportType(int)' would imply 'transportType' property.") ++ "property and 'com.ibm.mq.jms.MQConnectionFactory.setTransportType(int)' would imply 'transportType' property.", +expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY) @SeeAlso(classNames = {"org.apache.nifi.jms.processors.ConsumeJMS", "org.apache.nifi.jms.processors.PublishJMS"}) public class JMSConnectionFactoryProvider extends AbstractControllerService implements JMSConnectionFactoryProviderDefinition { @@ -134,6 +135,7 @@ public class JMSConnectionFactoryProvider extends AbstractControllerService impl + "' property to be set on the provided ConnectionFactory implementation.") .name(propertyDescriptorName) .addValidator(StandardValidators.NON_EMPTY_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) .dynamic(true) .build(); } diff --git a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java index b50a3d4..867db0d 100644 --- a/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java +++ b/nifi-nar-bundles/nifi-jms-bundle/nifi-jms-processors/src/test/java/org/apache/nifi/jms/cf/JMSConnectionFactoryProviderTest.java @@ -480,6 +480,25 @@ public class JMSConnectionFactoryProviderTest { runner.enableControllerService(cfProvider); -assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", HOSTNAME+"("+PORT+")")); +assertEquals(cfProvider.getSetProperties(), ImmutableMap.of("connectionNameList", HOSTNAME + "(" + PORT + ")")); +} + +@Test +public void dynamicPropertiesSetOnSingleTestBrokerConnectionFactory() throws InitializationException { +TestRunner runner = TestRunners.newTestRunner(mock(Processor.class)); + +JMSConnectionFactoryProviderForTest cfProvider = new JMSConnectionFactoryProviderForTest(); +runner.addControllerService(controllerServiceId, cfProvider); + +runner.setVariable("test", "dynamicValue"); + +runner.setProperty(cfProvider, JMSConnectionFactoryProvider.BROKER_URI, SINGLE_TEST_BROKER); +runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CLIENT_LIB_DIR_PATH, dummyResource); +runner.setProperty(cfProvider, JMSConnectionFactoryProvider.CONNECTION_FACTORY_IMPL, TEST_CONNECTION_FACTORY_IMPL); +runner.setProperty(cfProvider, "dynamicProperty", &qu
[nifi] branch master updated: NIFI-6778 - Added rules engine service api and Easy Rules Implementation
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 7c9d34f NIFI-6778 - Added rules engine service api and Easy Rules Implementation 7c9d34f is described below commit 7c9d34f820ea063349f7405c31d5587a040eb779 Author: Yolanda M. Davis AuthorDate: Wed Oct 16 10:49:37 2019 -0400 NIFI-6778 - Added rules engine service api and Easy Rules Implementation NIFI-6778 - added comments and component description annotations NIFI-6778 - added license and notice details NIFI-6778 - added additional details section and updated description of service NIFI-6778 - fixed checkstyle error NIFI-6778 - addressed review comments for documentation and description corrections NIFI-6778 - added include-rules profile for assembly build NIFI-6778 - add MVEL and ASM to license and notice NIFI-6778 - switch to use Jackson for NiFi Rules Json deserialization Signed-off-by: Matthew Burgess This closes #3824 --- nifi-assembly/LICENSE | 51 nifi-assembly/pom.xml | 16 +- .../nifi-easyrules-nar/pom.xml | 46 .../src/main/resources/META-INF/LICENSE| 258 + .../src/main/resources/META-INF/NOTICE | 46 .../nifi-easyrules-service/pom.xml | 96 .../java/org/apache/nifi/rules/RulesFactory.java | 127 ++ .../nifi/rules/engine/EasyRulesEngineService.java | 194 .../org.apache.nifi.controller.ControllerService | 15 ++ .../additionalDetails.html | 113 + .../org/apache/nifi/rules/TestRulesFactory.java| 136 +++ .../rules/engine/TestEasyRulesEngineService.java | 147 .../apache/nifi/rules/engine/TestProcessor.java| 46 .../src/test/resources/test_mvel_rules.json| 16 ++ .../src/test/resources/test_mvel_rules.yml | 14 ++ .../src/test/resources/test_nifi_rules.json| 34 +++ .../src/test/resources/test_nifi_rules.yml | 28 +++ .../src/test/resources/test_spel_rules.json| 16 ++ .../src/test/resources/test_spel_rules.yml | 14 ++ nifi-nar-bundles/nifi-easyrules-bundle/pom.xml | 42 .../nifi-rules-engine-service-api/pom.xml | 36 +++ .../main/java/org/apache/nifi/rules/Action.java| 53 + .../java/org/apache/nifi/rules/ActionHandler.java | 31 +++ .../src/main/java/org/apache/nifi/rules/Rule.java | 93 .../nifi/rules/engine/RulesEngineService.java | 47 .../nifi-standard-services-api-nar/pom.xml | 6 + nifi-nar-bundles/nifi-standard-services/pom.xml| 1 + nifi-nar-bundles/pom.xml | 9 +- 28 files changed, 1729 insertions(+), 2 deletions(-) diff --git a/nifi-assembly/LICENSE b/nifi-assembly/LICENSE index 66c7c13..a99d16d 100644 --- a/nifi-assembly/LICENSE +++ b/nifi-assembly/LICENSE @@ -2834,3 +2834,54 @@ The binary distribution of this product bundles 'JUnit Platform' and 'JUnit Jupi notice. You may add additional accurate notices of copyright ownership. + +The binary distribution of this product bundles 'easy-rules' under an MIT style license. + +Copyright (c) 2019 Mahmoud Ben Hassine (mahmoud.benhass...@icloud.com) + +Permission is hereby granted, free of charge, to any person obtaining a copy +of this software and associated documentation files (the "Software"), to deal +in the Software without restriction, including without limitation the rights +to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +copies of the Software, and to permit persons to whom the Software is +furnished to do so, subject to the following conditions: + +The above copyright notice and this permission notice shall be included in +all copies or substantial portions of the Software. + +THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +THE SOFTWARE. + +The binary distribution of this product bundles the 'ASM' library which is available under a BSD style license. + + Copyright (c) 2000-2005 INRIA, France Telecom + All rights reserved. + + Redistribution and use in source and binary forms, with or without + modification, are permitted provide
[nifi] branch master updated: NIFI-6801 - fix to ensure unique model instance is created for each connection. Also increased default query interval.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 1473652 NIFI-6801 - fix to ensure unique model instance is created for each connection. Also increased default query interval. 1473652 is described below commit 147365285cbe385a85e58371dd9570dc01683936 Author: Yolanda M. Davis AuthorDate: Wed Oct 23 10:52:26 2019 -0400 NIFI-6801 - fix to ensure unique model instance is created for each connection. Also increased default query interval. Signed-off-by: Matthew Burgess This closes #3838 --- .../org/apache/nifi/controller/FlowController.java | 8 ++ .../CachingConnectionStatusAnalyticsEngine.java| 6 ++--- .../analytics/ConnectionStatusAnalytics.java | 2 +- .../analytics/ConnectionStatusAnalyticsEngine.java | 7 ++--- .../analytics/StatusAnalyticsModelMapFactory.java | 30 +- ...TestCachingConnectionStatusAnalyticsEngine.java | 10 +++- .../TestConnectionStatusAnalyticsEngine.java | 7 ++--- .../analytics/TestStatusAnalyticsEngine.java | 11 +--- .../TestStatusAnalyticsModelMapFactory.java| 5 ++-- .../nifi-framework/nifi-resources/pom.xml | 2 +- 10 files changed, 42 insertions(+), 46 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index a6142ec..86cfab9 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -152,9 +152,7 @@ import org.apache.nifi.controller.state.manager.StandardStateManagerProvider; import org.apache.nifi.controller.state.server.ZooKeeperStateServer; import org.apache.nifi.controller.status.analytics.CachingConnectionStatusAnalyticsEngine; import org.apache.nifi.controller.status.analytics.StatusAnalyticsEngine; -import org.apache.nifi.controller.status.analytics.StatusAnalyticsModel; import org.apache.nifi.controller.status.analytics.StatusAnalyticsModelMapFactory; -import org.apache.nifi.controller.status.analytics.StatusMetricExtractFunction; import org.apache.nifi.controller.status.history.ComponentStatusRepository; import org.apache.nifi.controller.status.history.GarbageCollectionHistory; import org.apache.nifi.controller.status.history.GarbageCollectionStatus; @@ -214,7 +212,6 @@ import org.apache.nifi.util.ComponentIdGenerator; import org.apache.nifi.util.FormatUtils; import org.apache.nifi.util.NiFiProperties; import org.apache.nifi.util.ReflectionUtils; -import org.apache.nifi.util.Tuple; import org.apache.nifi.util.concurrency.TimedLock; import org.apache.nifi.web.api.dto.PositionDTO; import org.apache.nifi.web.api.dto.status.StatusHistoryDTO; @@ -648,10 +645,9 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node modelScoreThreshold = NiFiProperties.DEFAULT_ANALYTICS_CONNECTION_SCORE_THRESHOLD; } -final Map> modelMap = StatusAnalyticsModelMapFactory -.getConnectionStatusModelMap(extensionManager, nifiProperties); +StatusAnalyticsModelMapFactory statusAnalyticsModelMapFactory = new StatusAnalyticsModelMapFactory(extensionManager, nifiProperties); -analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, modelMap, +analyticsEngine = new CachingConnectionStatusAnalyticsEngine(flowManager, componentStatusRepository, flowFileEventRepository, statusAnalyticsModelMapFactory, predictionIntervalMillis, queryIntervalMillis, modelScoreName, modelScoreThreshold); } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java index aa67811..49e8ebe 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/CachingConnectionStatusAnalyticsEngine.java @@ -16,13 +16,11 @@ */ pack
[nifi] branch master updated: NIFI-6855 - added action type enforcement option for handlers
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git View the commit online: https://github.com/apache/nifi/commit/18245a4441406d7e4295997cba4369a27834dafd The following commit(s) were added to refs/heads/master by this push: new 18245a4 NIFI-6855 - added action type enforcement option for handlers 18245a4 is described below commit 18245a4441406d7e4295997cba4369a27834dafd Author: Yolanda M. Davis AuthorDate: Wed Nov 13 10:49:29 2019 -0500 NIFI-6855 - added action type enforcement option for handlers NIFI-6855 - added setting to support ignore, warn or throwing exception for unsupported action types. added EL support for defining types to enforce. NIFI-6855 - fix checkstyle violation Signed-off-by: Matthew Burgess This closes #3886 --- .../handlers/AbstractActionHandlerService.java | 94 +- .../apache/nifi/rules/handlers/AlertHandler.java | 13 +- .../nifi/rules/handlers/ExpressionHandler.java | 11 +- .../org/apache/nifi/rules/handlers/LogHandler.java | 12 +- .../nifi/rules/handlers/RecordSinkHandler.java | 16 ++- .../nifi/rules/handlers/TestAlertHandler.java | 127 ++- .../nifi/rules/handlers/TestExpressionHandler.java | 98 +++ .../apache/nifi/rules/handlers/TestLogHandler.java | 135 + 8 files changed, 488 insertions(+), 18 deletions(-) diff --git a/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AbstractActionHandlerService.java b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AbstractActionHandlerService.java index 57bbf28..4580ba0 100644 --- a/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AbstractActionHandlerService.java +++ b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-service/src/main/java/org/apache/nifi/rules/handlers/AbstractActionHandlerService.java @@ -16,22 +16,110 @@ */ package org.apache.nifi.rules.handlers; +import org.apache.commons.lang3.StringUtils; +import org.apache.nifi.annotation.lifecycle.OnEnabled; +import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.context.PropertyContext; import org.apache.nifi.controller.AbstractControllerService; +import org.apache.nifi.controller.ConfigurationContext; +import org.apache.nifi.expression.ExpressionLanguageScope; +import org.apache.nifi.processor.util.StandardValidators; +import org.apache.nifi.reporting.InitializationException; import org.apache.nifi.rules.Action; import org.apache.nifi.rules.PropertyContextActionHandler; +import java.util.Arrays; +import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public abstract class AbstractActionHandlerService extends AbstractControllerService implements PropertyContextActionHandler { -public static enum DebugLevels { +protected List enforceActionTypes; +protected EnforceActionTypeLevel enforceActionTypeLevel; + +public enum DebugLevels { trace, debug, info, warn, error } -public abstract void execute(Action action, Map facts); + +public enum EnforceActionTypeLevel { +IGNORE, WARN, EXCEPTION +} + +public static final PropertyDescriptor ENFORCE_ACTION_TYPE = new PropertyDescriptor.Builder() +.name("action-handler-enforce-type") +.displayName("Enforce Action Type") +.required(false) +.description("The Action Type(s) that should be supported by this handler. If provided any other type an " + +"exception will be thrown. This can support a comma delimited list of types (e.g. ALERT,LOG)") +.addValidator(StandardValidators.NON_BLANK_VALIDATOR) + .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY) +.build(); + +public static final PropertyDescriptor ENFORCE_ACTION_TYPE_LEVEL = new PropertyDescriptor.Builder() +.name("action-handler-enforce-type-level") +.displayName("Enforce Level") +.required(false) +.description("If specific action types are enforced, this setting specifies whether the action should be ignored," + +" a warning should be logged or if an exception is thrown. Default is to ignore the received action.") +.addValidator(StandardValidators.NON_BLANK_VALIDATOR) +.allowableValues(EnforceActionTypeLevel.values()) +.defaultValue("IGNORE") +.build(); + +public void execute(Action action, Map facts) {
[nifi] branch master updated: NIFI-6803 - Initial commit for ActionHandler Controller Services
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 5b28f6d NIFI-6803 - Initial commit for ActionHandler Controller Services 5b28f6d is described below commit 5b28f6dad9ea507dd1419ae4ed4e5fff8032b83d Author: Yolanda M. Davis AuthorDate: Tue Oct 29 16:21:45 2019 -0400 NIFI-6803 - Initial commit for ActionHandler Controller Services NIFI-6803 - updated to description NIFI-6803 - add Spring reference to Notice Signed-off-by: Matthew Burgess This closes #3856 --- nifi-assembly/pom.xml | 6 + .../nifi-rules-action-handler-nar/pom.xml | 47 .../src/main/resources/LICENSE | 236 + .../src/main/resources/NOTICE | 18 ++ .../nifi-rules-action-handler-service/pom.xml | 89 .../handlers/AbstractActionHandlerService.java | 37 .../nifi/rules/handlers/ActionHandlerLookup.java | 124 +++ .../nifi/rules/handlers/ExpressionHandler.java | 121 +++ .../org/apache/nifi/rules/handlers/LogHandler.java | 159 ++ .../nifi/rules/handlers/RecordSinkHandler.java | 144 + .../org.apache.nifi.controller.ControllerService | 18 ++ .../nifi/rules/handlers/MockComponentLog.java | 231 .../rules/handlers/TestActionHandlerLookup.java| 125 +++ .../nifi/rules/handlers/TestExpressionHandler.java | 150 + .../apache/nifi/rules/handlers/TestLogHandler.java | 150 + .../apache/nifi/rules/handlers/TestProcessor.java | 46 .../nifi/rules/handlers/TestRecordSinkHandler.java | 148 + .../nifi-rules-action-handler-bundle/pom.xml | 43 .../nifi/rules/PropertyContextActionHandler.java | 28 +++ nifi-nar-bundles/pom.xml | 3 +- 20 files changed, 1922 insertions(+), 1 deletion(-) diff --git a/nifi-assembly/pom.xml b/nifi-assembly/pom.xml index 50f15aa..3140bab 100755 --- a/nifi-assembly/pom.xml +++ b/nifi-assembly/pom.xml @@ -857,6 +857,12 @@ language governing permissions and limitations under the License. --> 1.10.0-SNAPSHOT nar + +org.apache.nifi +nifi-rules-action-handler-nar +1.10.0-SNAPSHOT +nar + diff --git a/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/pom.xml b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/pom.xml new file mode 100644 index 000..b22a195 --- /dev/null +++ b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/pom.xml @@ -0,0 +1,47 @@ + +http://maven.apache.org/POM/4.0.0; + xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance; + xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd;> + + +nifi-rules-action-handler-bundle +org.apache.nifi +1.10.0-SNAPSHOT + +4.0.0 + +nifi-rules-action-handler-nar +1.10.0-SNAPSHOT +nar + +true +true + + + +org.apache.nifi +nifi-rules-action-handler-service +1.10.0-SNAPSHOT + + +org.apache.nifi +nifi-standard-services-api-nar +1.10.0-SNAPSHOT +nar + + + + \ No newline at end of file diff --git a/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/src/main/resources/LICENSE b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/src/main/resources/LICENSE new file mode 100644 index 000..1f84df0 --- /dev/null +++ b/nifi-nar-bundles/nifi-rules-action-handler-bundle/nifi-rules-action-handler-nar/src/main/resources/LICENSE @@ -0,0 +1,236 @@ + + Apache License + Version 2.0, January 2004 +http://www.apache.org/licenses/ + + TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION + + 1. Definitions. + + "License" shall mean the terms and conditions for use, reproduction, + and distribution as defined by Sections 1 through 9 of this document. + + "Licensor" shall mean the copyright owner or entity authorized by + the copyright owner that is granting the License. + + "Legal Entity" shall mean the union of the acting entity and all + other entities that control, are controlled by, or are under common + control with that entity. For the purposes of this definition
[nifi] branch master updated: NIFI-5970 Handle multiple input FlowFiles at Put.initConnection
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new b3880a4 NIFI-5970 Handle multiple input FlowFiles at Put.initConnection b3880a4 is described below commit b3880a4a067915c56aa3d1b602717eab7ffa02fb Author: Koji Kawamura AuthorDate: Wed Jul 17 11:28:37 2019 +0900 NIFI-5970 Handle multiple input FlowFiles at Put.initConnection Signed-off-by: Matthew Burgess This closes #3583 --- .../processor/util/pattern/PartialFunctions.java | 2 +- .../apache/nifi/processor/util/pattern/Put.java| 4 +- .../org/apache/nifi/processors/hive/PutHiveQL.java | 4 +- .../apache/nifi/processors/hive/PutHive3QL.java| 2 +- .../apache/nifi/processors/hive/PutHive_1_1QL.java | 4 +- .../processors/standard/PutDatabaseRecord.java | 4 +- .../apache/nifi/processors/standard/PutSQL.java| 31 -- .../nifi/processors/standard/TestPutSQL.java | 71 ++ .../java/org/apache/nifi/dbcp/DBCPService.java | 47 ++ .../apache/nifi/dbcp/DBCPConnectionPoolLookup.java | 20 +- .../nifi/dbcp/TestDBCPConnectionPoolLookup.java| 51 11 files changed, 223 insertions(+), 17 deletions(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/PartialFunctions.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/PartialFunctions.java index 7b969b0..9c27200 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/PartialFunctions.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/PartialFunctions.java @@ -32,7 +32,7 @@ public class PartialFunctions { @FunctionalInterface public interface InitConnection { -C apply(ProcessContext context, ProcessSession session, FC functionContext, FlowFile flowFile) throws ProcessException; +C apply(ProcessContext context, ProcessSession session, FC functionContext, List flowFiles) throws ProcessException; } @FunctionalInterface diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/Put.java b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/Put.java index 80b8088..bcea833 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/Put.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-processor-utils/src/main/java/org/apache/nifi/processor/util/pattern/Put.java @@ -93,8 +93,8 @@ public class Put { return; } -// Only pass in a flow file if there is a single one present -try (C connection = initConnection.apply(context, session, functionContext, flowFiles.size() == 1 ? flowFiles.get(0) : null)) { +// Pass the FlowFiles to initialize a connection +try (C connection = initConnection.apply(context, session, functionContext, flowFiles)) { try { // Execute the core function. diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java index 943e288..e1aeade 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java @@ -204,9 +204,9 @@ public class PutHiveQL extends AbstractHiveQLProcessor { } } -private InitConnection initConnection = (context, session, fc, ff) -> { +private InitConnection initConnection = (context, session, fc, ffs) -> { final HiveDBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(HiveDBCPService.class); -final Connection connection = dbcpService.getConnection(ff == null ? Collections.emptyMap() : ff.getAttributes()); +final Connection connection = dbcpService.getConnection(ffs == null || ffs.isEmpty() ? Collections.emptyMap() : ffs.get(0).getAttributes()); fc.connectionUrl = dbcpService.getConnectionURL(); return connection; }; diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3QL.java index 7a5b389..7
[nifi] branch master updated: NIFI-6753: Fixed bug where all values being provided to the CSV Writer were String objects, which resulted in the CSV Writer improperly quoting numeric values when the sc
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new e394f66 NIFI-6753: Fixed bug where all values being provided to the CSV Writer were String objects, which resulted in the CSV Writer improperly quoting numeric values when the schema indicates that the value is a number. Now, we will only convert the value to a String if the value is not a Number and/or the schema does not indicate a numeric ty type e394f66 is described below commit e394f6683aaadc20f59ab467ab781e50c86ba958 Author: Mark Payne AuthorDate: Wed Oct 9 09:22:18 2019 -0400 NIFI-6753: Fixed bug where all values being provided to the CSV Writer were String objects, which resulted in the CSV Writer improperly quoting numeric values when the schema indicates that the value is a number. Now, we will only convert the value to a String if the value is not a Number and/or the schema does not indicate a numeric ty type Signed-off-by: Matthew Burgess This closes #3797 --- .../java/org/apache/nifi/csv/WriteCSVResult.java | 40 +- .../org/apache/nifi/csv/TestWriteCSVResult.java| 37 2 files changed, 68 insertions(+), 9 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/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 020b7b1..6526ebe 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 @@ -17,14 +17,6 @@ package org.apache.nifi.csv; -import java.io.IOException; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Optional; -import java.util.Set; - import org.apache.commons.csv.CSVFormat; import org.apache.commons.csv.CSVPrinter; import org.apache.nifi.schema.access.SchemaAccessWriter; @@ -35,8 +27,17 @@ import org.apache.nifi.serialization.record.DataType; import org.apache.nifi.serialization.record.RawRecordWriter; 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 java.io.IOException; +import java.io.OutputStream; +import java.io.OutputStreamWriter; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + public class WriteCSVResult extends AbstractRecordSetWriter implements RecordSetWriter, RawRecordWriter { private final RecordSchema recordSchema; private final SchemaAccessWriter schemaWriter; @@ -142,13 +143,34 @@ public class WriteCSVResult extends AbstractRecordSetWriter implements RecordSet int i = 0; for (final RecordField recordField : recordSchema.getFields()) { -fieldValues[i++] = record.getAsString(recordField, getFormat(recordField)); +fieldValues[i++] = getFieldValue(record, recordField); } printer.printRecord(fieldValues); return schemaWriter.getAttributes(recordSchema); } +private Object getFieldValue(final Record record, final RecordField recordField) { +final RecordFieldType fieldType = recordField.getDataType().getFieldType(); + +switch (fieldType) { +case BIGINT: +case BYTE: +case DOUBLE: +case FLOAT: +case LONG: +case INT: +case SHORT: +final Object value = record.getValue(recordField); +if (value instanceof Number) { +return value; +} +break; +} + +return record.getAsString(recordField, getFormat(recordField)); +} + @Override public WriteResult writeRawRecord(final Record record) throws IOException { // If we are not writing an active record set, then we need to ensure that we write the 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
[nifi] branch master updated: NIFI-6723: Enrich processor-related and JVM GC metrics in Prometheus Reporting Task
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new d7ca37d NIFI-6723: Enrich processor-related and JVM GC metrics in Prometheus Reporting Task d7ca37d is described below commit d7ca37d065677e3016477f74ecdfe233ccd725c7 Author: Kotaro Terada AuthorDate: Fri Sep 27 09:53:39 2019 +0900 NIFI-6723: Enrich processor-related and JVM GC metrics in Prometheus Reporting Task Signed-off-by: Matthew Burgess This closes #3771 --- .../prometheus/api/PrometheusMetricsUtil.java | 63 +++--- 1 file changed, 56 insertions(+), 7 deletions(-) diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java index 74b0ccc..fec3338 100644 --- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java +++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/src/main/java/org/apache/nifi/reporting/prometheus/api/PrometheusMetricsUtil.java @@ -20,6 +20,7 @@ package org.apache.nifi.reporting.prometheus.api; import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.Map; +import java.util.concurrent.TimeUnit; import io.prometheus.client.SimpleCollector; import org.apache.nifi.components.AllowableValue; @@ -47,7 +48,7 @@ public class PrometheusMetricsUtil { private static final CollectorRegistry NIFI_REGISTRY = new CollectorRegistry(); private static final CollectorRegistry JVM_REGISTRY = new CollectorRegistry(); -// Process Group metrics +// Processor / Process Group metrics private static final Gauge AMOUNT_FLOWFILES_SENT = Gauge.build() .name("nifi_amount_flowfiles_sent") .help("Total number of FlowFiles sent by the component") @@ -66,6 +67,12 @@ public class PrometheusMetricsUtil { .labelNames("instance", "component_type", "component_name", "component_id", "parent_id") .register(NIFI_REGISTRY); +private static final Gauge AMOUNT_FLOWFILES_REMOVED = Gauge.build() +.name("nifi_amount_flowfiles_removed") +.help("Total number of FlowFiles removed by the component") +.labelNames("instance", "component_type", "component_name", "component_id", "parent_id") +.register(NIFI_REGISTRY); + private static final Gauge AMOUNT_BYTES_SENT = Gauge.build() .name("nifi_amount_bytes_sent") .help("Total number of bytes sent by the component") @@ -150,6 +157,7 @@ public class PrometheusMetricsUtil { "source_id", "source_name", "destination_id", "destination_name") .register(NIFI_REGISTRY); +// Processor metrics private static final Gauge PROCESSOR_COUNTERS = Gauge.build() .name("nifi_processor_counters") .help("Counters exposed by NiFi Processors") @@ -252,6 +260,18 @@ public class PrometheusMetricsUtil { .labelNames("instance") .register(JVM_REGISTRY); +private static final Gauge JVM_GC_RUNS = Gauge.build() +.name("nifi_jvm_gc_runs") +.help("NiFi JVM GC number of runs") +.labelNames("instance", "gc_name") +.register(JVM_REGISTRY); + +private static final Gauge JVM_GC_TIME = Gauge.build() +.name("nifi_jvm_gc_time") +.help("NiFi JVM GC time in milliseconds") +.labelNames("instance", "gc_name") +.register(JVM_REGISTRY); + public static CollectorRegistry createNifiMetrics(ProcessGroupStatus status, String instanceId, String parentPGId, String componentType, String metricsStrategy) { final String componentId = status.getId(); @@ -305,10 +325,10 @@ public class PrometheusMetricsUtil { if (METRICS_STRATEGY_COMPONENTS.getValue().equals(metricsStrategy)) { // Report metrics for all components -for(ProcessorStatus processorStatus : status.getProcessorStatus()) { +for (ProcessorStatus processorStatus : status.getProcessorStatus()) { Map counters = processorStatus.getCounters(); -if(counters != null) { +if (counters != nu
[nifi] branch master updated: NIFI-7109 Unit tests should be able to determine if item validator was called - Create a mock Validator to track count of calls to validate(). We cannot use Mockito for t
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 6e8f10c NIFI-7109 Unit tests should be able to determine if item validator was called - Create a mock Validator to track count of calls to validate(). We cannot use Mockito for this, because it can't mock all the StandardValidators 6e8f10c is described below commit 6e8f10c4f66f0167b9a43c91c549f1d36208d9cc Author: Otto Fowler AuthorDate: Thu Feb 6 10:07:49 2020 -0500 NIFI-7109 Unit tests should be able to determine if item validator was called - Create a mock Validator to track count of calls to validate(). We cannot use Mockito for this, because it can't mock all the StandardValidators refactor based on review comments fix naming in comments moved to main based on review Signed-off-by: Matthew Burgess This closes #4043 --- .../validator/InstrumentedStandardValidator.java | 92 ++ .../util/validator/TestStandardValidators.java | 73 + .../nifi/util/validator/InstrumentedValidator.java | 92 ++ 3 files changed, 243 insertions(+), 14 deletions(-) diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/InstrumentedStandardValidator.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/InstrumentedStandardValidator.java new file mode 100644 index 000..b5f6f39 --- /dev/null +++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/InstrumentedStandardValidator.java @@ -0,0 +1,92 @@ +/* + * 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.util.validator; + +import org.apache.nifi.components.ValidationContext; +import org.apache.nifi.components.ValidationResult; +import org.apache.nifi.components.Validator; + +/**: + * InstrumentedStandarValidator wraps a {@class Validator} and provides statistics on it's interactions. + * Because many of the {@class Validator} instances returned from {@class StandardValidator } + * are not mockable with with mockito, this is required to know, when running a test, if a + * {@class Validator} was in fact called, for example. + */ +public class InstrumentedStandardValidator implements Validator { + +/** + * Flag to reset a count after retrieving it. + * Thus not having to explicitly call reset() for simple cases. + */ +boolean doReset = false; + +/** + * Count the number of calls to validate() + */ +private int validateCallCount; +private Validator mockedValidator; + +/** + * Constructs a new {@class InstrumentedStandarValidator}. + * + * @param mockedValidator the {@class Validator} to wrap. + */ +public InstrumentedStandardValidator(Validator mockedValidator) { +this(mockedValidator,false); +} + +/** + * Constructs a new {@class InstrumentedStandarValidator}. + * + * @param mockedValidator the {@class Validator} to wrap. + */ +public InstrumentedStandardValidator(Validator mockedValidator, boolean resetOnGet) { +this.mockedValidator = mockedValidator; +this.doReset = resetOnGet; +} + +/** + * Default constructor without wrapping not supported. + * + */ +private InstrumentedStandardValidator(){} + +@Override +public ValidationResult validate(String subject, String input, ValidationContext context) { +validateCallCount++; +return mockedValidator.validate(subject, input, context); +} + +/** + * Returns the count of times validate was called + * @return count of validate() calls + */ +public int getValidateCallCount() { +int count = validateCallCount; +if (doReset) { +validateCallCount = 0; +} +return count; +} + +/** + * Resets the count of all calls to 0. + */ +public void resetAll() { +validateCallCount = 0; +} +} diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache
[nifi] branch master updated: NIFI-7007: Add update functionality to the PutCassandraRecord processor.
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 9848eb4 NIFI-7007: Add update functionality to the PutCassandraRecord processor. 9848eb4 is described below commit 9848eb4409fe672e7d321ea3ac3351e8a5972e14 Author: Wouter de Vries AuthorDate: Fri Jan 10 09:09:38 2020 +0100 NIFI-7007: Add update functionality to the PutCassandraRecord processor. NIFI-7007: Add additional unit tests that hit non-happy path NIFI-7007: Use AllowableValue instead of string NIFI-7007: Add the use of attributes for the update method, statement type and batch statement type NIFI-7007: Add additional tests, mainly for the use of attributes NIFI-7007: add some ReadsAttribute properties to the PutCassandraRecord processor NIFI-7007: additional update keys validation logic NIFI-7007: fix imports NIFI-7007: Convert fieldValue to long in separate method NIFI-7007: Add new style of tests checking actual CQL output NIFI-7007: add license to new test file NIFI-7007: add customValidate to check for certain incompatible property combinations NIFI-7007: remove check on updateMethod and replace Set.of with java 8 compatible replacmenet NIFI-7007: Add test for failure with empty update method via attributes NIFI-7007: remove unused variable NIFI-7007: Fix customValidate that incorrectly invalidated a valid config Fix Checkstyle Signed-off-by: Matthew Burgess This closes #3977 --- .../processors/cassandra/PutCassandraRecord.java | 259 ++- .../cassandra/PutCassandraRecordTest.java | 362 + .../cassandra/PutCassandraRecordUpdateTest.java| 293 + 3 files changed, 900 insertions(+), 14 deletions(-) diff --git a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java index 8acc36a..37fefa8 100644 --- a/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java +++ b/nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraRecord.java @@ -19,14 +19,23 @@ package org.apache.nifi.processors.cassandra; import com.datastax.driver.core.BatchStatement; import com.datastax.driver.core.ConsistencyLevel; import com.datastax.driver.core.Session; +import com.datastax.driver.core.Statement; +import com.datastax.driver.core.querybuilder.Assignment; import com.datastax.driver.core.querybuilder.Insert; import com.datastax.driver.core.querybuilder.QueryBuilder; +import com.datastax.driver.core.querybuilder.Update; +import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.ReadsAttribute; +import org.apache.nifi.annotation.behavior.ReadsAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnShutdown; import org.apache.nifi.annotation.lifecycle.OnUnscheduled; +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.expression.ExpressionLanguageScope; import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.processor.ProcessContext; @@ -34,8 +43,8 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; -import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.RecordReader; +import org.apache.nifi.serialization.RecordReaderFactory; import org.apache.nifi.serialization.record.Record; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; @@ -45,18 +54,63 @@ import org.apache.nifi.util.StopWatch; import java.io.InputStream; import java.util.Arrays; import java.util.Collections; +import java.util.Collection; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; + +import static java.lang.String.format; @Tags
[nifi] branch master updated: NIFI-5924 Labels should be searchable
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new acaf321 NIFI-5924 Labels should be searchable acaf321 is described below commit acaf321af022929533f0dc25e0e1eba796bc3e46 Author: MatthewKnight-NG AuthorDate: Fri Nov 22 15:14:22 2019 -0500 NIFI-5924 Labels should be searchable Signed-off-by: Matthew Burgess This closes #4070 --- .../nifi/web/api/dto/search/SearchResultsDTO.java | 15 .../web/controller/ControllerSearchService.java| 33 +++- .../controller/ControllerSearchServiceTest.java| 44 ++ .../nf-ng-canvas-flow-status-controller.js | 8 4 files changed, 98 insertions(+), 2 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java index 5054c29..1946261 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-client-dto/src/main/java/org/apache/nifi/web/api/dto/search/SearchResultsDTO.java @@ -35,6 +35,7 @@ public class SearchResultsDTO { private List outputPortResults = new ArrayList<>(); private List remoteProcessGroupResults = new ArrayList<>(); private List funnelResults = new ArrayList<>(); +private List labelResults = new ArrayList<>(); private List parameterContextResults = new ArrayList<>(); private List parameterResults = new ArrayList<>(); @@ -137,6 +138,20 @@ public class SearchResultsDTO { } /** + * @return labels that matched the search + */ +@ApiModelProperty( +value = "The labels that matched the search." +) +public List getLabelResults() { +return labelResults; +} + +public void setLabelResults(List labelResults) { +this.labelResults = labelResults; +} + +/** * @return parameter contexts that matched the search. */ @ApiModelProperty( diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java index 9bec4bb..a41288e 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-api/src/main/java/org/apache/nifi/web/controller/ControllerSearchService.java @@ -31,6 +31,7 @@ import org.apache.nifi.connectable.Port; import org.apache.nifi.controller.FlowController; import org.apache.nifi.controller.ProcessorNode; import org.apache.nifi.controller.ScheduledState; +import org.apache.nifi.controller.label.Label; import org.apache.nifi.controller.queue.FlowFileQueue; import org.apache.nifi.flowfile.FlowFilePrioritizer; import org.apache.nifi.groups.ProcessGroup; @@ -74,8 +75,8 @@ public class ControllerSearchService { * Searches term in the controller beginning from a given process group. * * @param results Search results - * @param search The search term - * @param group The init process group + * @param search The search term + * @param group The init process group */ public void search(final SearchResultsDTO results, final String search, final ProcessGroup group) { final NiFiUser user = NiFiUserUtils.getNiFiUser(); @@ -162,6 +163,18 @@ public class ControllerSearchService { } } +for (final Label label : group.getLabels()) { +if (label.isAuthorized(authorizer, RequestAction.READ, user)) { +final ComponentSearchResultDTO match = search(search, label); +if (match != null) { +match.setGroupId(group.getIdentifier()); +match.setParentGroup(buildResultGroup(group, user)); +match.setVersionedGroup(buildVersionedGroup(group, user)); +results.getLabelResults().add(match); +} +} +} + for (final ProcessGroup processGroup : group.getProcessGroups()) { search(results, search, processGroup); } @@ -511,6 +524,22 @@ public class ControllerSearchService { return dto; } +private ComponentSearchResultDTO sea
[nifi] branch master updated: NIFI-7139 Add record.error.message on failure of a record reader or writer
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 0bb8ce7 NIFI-7139 Add record.error.message on failure of a record reader or writer 0bb8ce7 is described below commit 0bb8ce7438d9855dcca6bf89e3a672d1f9477593 Author: Shawn Weeks AuthorDate: Thu Feb 13 08:39:49 2020 -0600 NIFI-7139 Add record.error.message on failure of a record reader or writer Handle scenario where message might be null. Update to test case that was failing because adding attributes modified a flow file even if you don't change the contents. Fixed Style Issues and Updated WritesAttributes. Added Test Case for Error Message Signed-off-by: Matthew Burgess This closes #4052 --- .../nifi/processors/standard/AbstractRecordProcessor.java | 9 + .../org/apache/nifi/processors/standard/ConvertRecord.java| 3 ++- .../org/apache/nifi/processors/standard/UpdateRecord.java | 6 +- .../apache/nifi/processors/standard/TestConvertRecord.java| 11 ++- 4 files changed, 22 insertions(+), 7 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java index 8ccea5a..1ea70e2 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractRecordProcessor.java @@ -174,6 +174,15 @@ public abstract class AbstractRecordProcessor extends AbstractProcessor { }); } catch (final Exception e) { getLogger().error("Failed to process {}; will route to failure", new Object[] {flowFile, e}); +// Since we are wrapping the exceptions above there should always be a cause +// but it's possible it might not have a message. This handles that by logging +// the name of the class thrown. +Throwable c = e.getCause(); +if (c != null) { +session.putAttribute(flowFile, "record.error.message", (c.getLocalizedMessage() != null) ? c.getLocalizedMessage() : c.getClass().getCanonicalName() + " Thrown"); +} else { +session.putAttribute(flowFile, "record.error.message", e.getClass().getCanonicalName() + " Thrown"); +} session.transfer(flowFile, REL_FAILURE); return; } 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 1be1794..a1e6f99 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 @@ -41,7 +41,8 @@ import java.util.List; @Tags({"convert", "record", "generic", "schema", "json", "csv", "avro", "log", "logs", "freeform", "text"}) @WritesAttributes({ @WritesAttribute(attribute = "mime.type", description = "Sets the mime.type attribute to the MIME Type specified by the Record Writer"), -@WritesAttribute(attribute = "record.count", description = "The number of records in the FlowFile") +@WritesAttribute(attribute = "record.count", description = "The number of records in the FlowFile"), +@WritesAttribute(attribute = "record.error.message", description = "This attribute provides on failure the error message encountered by the Reader or Writer.") }) @CapabilityDescription("Converts records from one data format to another using configured Record Reader and Record Write Controller Services. " + "The Reader and Writer must be configured with \"matching\" schemas. By this, we mean the schemas must have the same field names. The types of the fields " diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/UpdateRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache
[nifi] branch master updated: NIFI-7025: Wrap Hive 3 calls with UGI.doAs Updated PutHive3Streaming to wrap calls to Hive in UGI.doAs methods Fixed misleading logging message after the principal has be
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 4b6de8d NIFI-7025: Wrap Hive 3 calls with UGI.doAs Updated PutHive3Streaming to wrap calls to Hive in UGI.doAs methods Fixed misleading logging message after the principal has been authenticated with the KDC When connecting to unsecured Hive 3, a UGI with "simple" auth will be used 4b6de8d is described below commit 4b6de8d164a2fe52d03fe06e751e2ece4ce7c680 Author: jstorck AuthorDate: Tue Mar 3 20:03:59 2020 -0500 NIFI-7025: Wrap Hive 3 calls with UGI.doAs Updated PutHive3Streaming to wrap calls to Hive in UGI.doAs methods Fixed misleading logging message after the principal has been authenticated with the KDC When connecting to unsecured Hive 3, a UGI with "simple" auth will be used Signed-off-by: Matthew Burgess This closes #4108 --- .../nifi/processors/hive/PutHive3Streaming.java| 317 +++-- .../processors/hive/TestPutHive3Streaming.java | 12 +- 2 files changed, 184 insertions(+), 145 deletions(-) diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java index a1123d2..23b873f 100644 --- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java +++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-processors/src/main/java/org/apache/nifi/processors/hive/PutHive3Streaming.java @@ -67,8 +67,10 @@ import org.apache.nifi.util.hive.HiveOptions; import org.apache.nifi.util.hive.HiveUtils; import org.apache.nifi.util.hive.ValidationResources; +import javax.security.auth.login.LoginException; import java.io.IOException; import java.io.InputStream; +import java.security.PrivilegedAction; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -321,7 +323,7 @@ public class PutHive3Streaming extends AbstractProcessor { } @OnScheduled -public void setup(final ProcessContext context) { +public void setup(final ProcessContext context) throws IOException { ComponentLog log = getLogger(); rollbackOnFailure = context.getProperty(ROLLBACK_ON_FAILURE).asBoolean(); @@ -368,9 +370,9 @@ public class PutHive3Streaming extends AbstractProcessor { throw new ProcessException(ae); } -log.info("Successfully logged in as principal {} with keytab {}", new Object[]{resolvedPrincipal, resolvedKeytab}); +log.info("Successfully logged in as principal " + resolvedPrincipal); } else { -ugi = null; +ugi = SecurityUtil.loginSimple(hiveConfig); kerberosUserReference.set(null); } @@ -381,172 +383,181 @@ public class PutHive3Streaming extends AbstractProcessor { } public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { -FlowFile flowFile = session.get(); -if (flowFile == null) { -return; -} - -final RecordReaderFactory recordReaderFactory = context.getProperty(RECORD_READER).asControllerService(RecordReaderFactory.class); -final String dbName = context.getProperty(DB_NAME).evaluateAttributeExpressions(flowFile).getValue(); -final String tableName = context.getProperty(TABLE_NAME).evaluateAttributeExpressions(flowFile).getValue(); - -final ComponentLog log = getLogger(); -String metastoreURIs = null; -if (context.getProperty(METASTORE_URI).isSet()) { -metastoreURIs = context.getProperty(METASTORE_URI).evaluateAttributeExpressions(flowFile).getValue(); -if (StringUtils.isEmpty(metastoreURIs)) { -// Shouldn't be empty at this point, log an error, penalize the flow file, and return -log.error("The '" + METASTORE_URI.getDisplayName() + "' property evaluated to null or empty, penalizing flow file, routing to failure"); -session.transfer(session.penalize(flowFile), REL_FAILURE); +getUgi().doAs((PrivilegedAction) () -> { +FlowFile flowFile = session.get(); +if (flowFile == null) { +return null; } -} - -final String staticPartitionValuesString = context.getProperty(STATIC_PARTITION_VALUES).evaluateAttributeExpressions(flowFile).getValue(); -final boolean disableStreamingOptimizations = context.getProperty(DISABLE_STREAMING_OPTIMIZATIONS).asBoolean(); - -// Override the Hive Metastore URIs in t
[nifi] branch master updated: NIFI-7055 handle empty split evaluations, which contain only ,
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new f1c6e92 NIFI-7055 handle empty split evaluations, which contain only , f1c6e92 is described below commit f1c6e92df58bf24eb5199cdcb1784cbc438946db Author: Otto Fowler AuthorDate: Thu Jan 23 09:20:17 2020 -0500 NIFI-7055 handle empty split evaluations, which contain only , add explict test for " , " updated with counting validator Signed-off-by: Matthew Burgess This closes #4012 --- .../nifi/processor/util/StandardValidators.java | 5 + .../nifi/util/validator/TestStandardValidators.java | 19 +++ 2 files changed, 24 insertions(+) diff --git a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/processor/util/StandardValidators.java b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/processor/util/StandardValidators.java index 6bf9aaa..f2ca0a1 100644 --- a/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/processor/util/StandardValidators.java +++ b/nifi-commons/nifi-utils/src/main/java/org/apache/nifi/processor/util/StandardValidators.java @@ -575,7 +575,12 @@ public class StandardValidators { if (input == null) { return new ValidationResult.Builder().subject(subject).input(null).explanation("List must have at least one non-empty element").valid(false).build(); } + final String[] list = input.split(","); +if (list.length == 0) { +return new ValidationResult.Builder().subject(subject).input(null).explanation("List must have at least one non-empty element").valid(false).build(); +} + for (String item : list) { String itemToValidate = trimEntries ? item.trim() : item; if (!isEmpty(itemToValidate) || !excludeEmptyEntries) { diff --git a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/TestStandardValidators.java b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/TestStandardValidators.java index e469955..6c6bfe9 100644 --- a/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/TestStandardValidators.java +++ b/nifi-commons/nifi-utils/src/test/java/org/apache/nifi/util/validator/TestStandardValidators.java @@ -184,6 +184,25 @@ public class TestStandardValidators { assertFalse(vr.isValid()); assertEquals(1, mockValidator.getValidateCallCount()); +// An empty list is the same as null, "" or " " +vr = val.validate("List", ",", validationContext); +assertFalse(vr.isValid()); +assertEquals(0, mockValidator.getValidateCallCount()); + +vr = val.validate("List", " , ", validationContext); +assertFalse(vr.isValid()); +assertEquals(1, mockValidator.getValidateCallCount()); + +// will evaluate to no entry +vr = val.validate("List", "", validationContext); +assertFalse(vr.isValid()); +assertEquals(0, mockValidator.getValidateCallCount()); + +// will evaluate to an empty element +vr = val.validate("List", ",foo", validationContext); +assertFalse(vr.isValid()); +assertEquals(1, mockValidator.getValidateCallCount()); + vr = val.validate("List", "1", validationContext); assertTrue(vr.isValid()); assertEquals(1, mockValidator.getValidateCallCount());
[nifi] branch master updated: NIFI-7163 - added RulesEngine and RulesEngineProvider interfaces, enhanced easy rules to support provider interface and refactored to extract rules engine implementation
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new abf223d NIFI-7163 - added RulesEngine and RulesEngineProvider interfaces, enhanced easy rules to support provider interface and refactored to extract rules engine implementation abf223d is described below commit abf223d574f0d5c1d3484374535e6ae1d0f6c6c0 Author: Yolanda M. Davis AuthorDate: Mon Feb 24 15:27:39 2020 -0500 NIFI-7163 - added RulesEngine and RulesEngineProvider interfaces, enhanced easy rules to support provider interface and refactored to extract rules engine implementation NIFI-7163 - updated documentation and comments NIFI-7163 - fix checkstyle issues Signed-off-by: Matthew Burgess This closes #4081 --- .../nifi-easyrules-service/pom.xml | 6 + .../org/apache/nifi/rules/RulesMVELCondition.java | 3 +- .../org/apache/nifi/rules/RulesSPELCondition.java | 3 +- ...java => AbstractEasyRulesEngineController.java} | 114 +++- .../apache/nifi/rules/engine/EasyRulesEngine.java | 178 +++ .../nifi/rules/engine/EasyRulesEngineProvider.java | 37 .../nifi/rules/engine/EasyRulesEngineService.java | 191 +++-- .../org.apache.nifi.controller.ControllerService | 3 +- .../additionalDetails.html | 5 +- .../org/apache/nifi/rules/TestRulesCondition.java | 100 +++ .../nifi/rules/engine/TestEasyRulesEngine.java | 87 ++ .../rules/engine/TestEasyRulesEngineProvider.java | 59 +++ .../rules/engine/TestEasyRulesEngineService.java | 25 ++- .../src/test/resources/test_nifi_rules_filter.json | 34 .../main/java/org/apache/nifi/rules/Action.java| 12 +- .../src/main/java/org/apache/nifi/rules/Rule.java | 24 ++- .../rules/{Action.java => engine/RulesEngine.java} | 40 ++--- ...EngineService.java => RulesEngineProvider.java} | 21 +-- .../nifi/rules/engine/RulesEngineService.java | 2 +- 19 files changed, 626 insertions(+), 318 deletions(-) diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml index c36cfe4..9a0ae5a 100644 --- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/pom.xml @@ -50,6 +50,11 @@ 3.3.0 +org.apache.commons +commons-jexl3 +3.1 + + org.jeasy easy-rules-spel 3.3.0 @@ -76,6 +81,7 @@ src/test/resources/test_nifi_rules.json src/test/resources/test_nifi_rules.yml + src/test/resources/test_nifi_rules_filter.json src/test/resources/test_mvel_rules.json src/test/resources/test_mvel_rules.yml src/test/resources/test_spel_rules.json diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java index 7bd615f..969d445 100644 --- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesMVELCondition.java @@ -19,7 +19,6 @@ package org.apache.nifi.rules; import org.jeasy.rules.api.Condition; import org.jeasy.rules.api.Facts; -import org.jeasy.rules.mvel.MVELCondition; import org.mvel2.MVEL; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -28,7 +27,7 @@ import java.io.Serializable; public class RulesMVELCondition implements Condition { -private static final Logger LOGGER = LoggerFactory.getLogger(MVELCondition.class); +private static final Logger LOGGER = LoggerFactory.getLogger(RulesMVELCondition.class); private String expression; private Serializable compiledExpression; private boolean ignoreConditionErrors; diff --git a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java index e753f93..881da6b 100644 --- a/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java +++ b/nifi-nar-bundles/nifi-easyrules-bundle/nifi-easyrules-service/src/main/java/org/apache/nifi/rules/RulesSPELCondition.java
[nifi] branch master updated: NIFI-4792: Add support for querying array fields in QueryRecord
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 5e964fb NIFI-4792: Add support for querying array fields in QueryRecord 5e964fb is described below commit 5e964fbc474034cb5342e22aa290f1040af377ea Author: David Savage AuthorDate: Sat Jan 25 17:27:47 2020 + NIFI-4792: Add support for querying array fields in QueryRecord Work in progress adding support for array based queries updated calcite dependency tidy up unused imports highlighted by checkstyle in travis build tidy up }s highlighted by checkstyle in travis build Add test for use case referenced in NIFI-4792 Bumped Calcite version to 1.21.0 Signed-off-by: Matthew Burgess This closes #4015 --- .../nifi/processors/standard/QueryRecord.java | 36 - .../nifi/queryrecord/FlowFileEnumerator.java | 19 ++- .../org/apache/nifi/queryrecord/FlowFileTable.java | 5 +- .../nifi/processors/standard/TestQueryRecord.java | 158 + nifi-nar-bundles/nifi-standard-bundle/pom.xml | 2 +- 5 files changed, 214 insertions(+), 6 deletions(-) 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 5136e67..82aea6f 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 @@ -610,14 +610,15 @@ public class QueryRecord extends AbstractProcessor { if (record == null) { return null; } - if (record instanceof Record) { return eval((Record) record, recordPath); } if (record instanceof Record[]) { return eval((Record[]) record, recordPath); } - +if (record instanceof Iterable) { +return eval((Iterable) record, recordPath); +} if (record instanceof Map) { return eval((Map) record, recordPath); } @@ -645,6 +646,18 @@ public class QueryRecord extends AbstractProcessor { return evalResults(selectedFields); } +private Object eval(final Iterable records, final String recordPath) { +final RecordPath compiled = RECORD_PATH_CACHE.getCompiled(recordPath); + +final List selectedFields = new ArrayList<>(); +for (final Record record : records) { +final RecordPathResult result = compiled.evaluate(record); +result.getSelectedFields().forEach(selectedFields::add); +} + +return evalResults(selectedFields); +} + private Object eval(final Record[] records, final String recordPath) { final RecordPath compiled = RECORD_PATH_CACHE.getCompiled(recordPath); @@ -794,6 +807,8 @@ public class QueryRecord extends AbstractProcessor { return eval((Record) record, recordPath, transform); } else if (record instanceof Record[]) { return eval((Record[]) record, recordPath, transform); +} else if (record instanceof Iterable) { +return eval((Iterable) record, recordPath, transform); } else if (record instanceof Map) { return eval((Map) record, recordPath, transform); } @@ -837,6 +852,23 @@ public class QueryRecord extends AbstractProcessor { return evalResults(selectedFields.stream(), transform, () -> "RecordPath " + recordPath + " resulted in more than one return value. The RecordPath must be further constrained."); } +private T eval(final Iterable records, final String recordPath, final Function transform) { +final RecordPath compiled = RECORD_PATH_CACHE.getCompiled(recordPath); + +final List selectedFields = new ArrayList<>(); +for (final Record record : records) { +final RecordPathResult result = compiled.evaluate(record); +result.getSelectedFields().forEach(selectedFields::add); +} + +if (selectedFields.isEmpty()) { +return null; +} + +return evalResults(selectedFields.stream(), transform, () -> "RecordPath " + recordPath + " resulted in more than one return value. The RecordPath must be further constrained."); +} + + p
[nifi] branch master updated: NIFI-7106 - Add parent name and parent path in SiteToSiteStatusReportingTask
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 58bcd6c NIFI-7106 - Add parent name and parent path in SiteToSiteStatusReportingTask 58bcd6c is described below commit 58bcd6c5ddc1989e99b5630b89f413ef4726b4a0 Author: Pierre Villard AuthorDate: Tue Feb 4 22:35:11 2020 -0500 NIFI-7106 - Add parent name and parent path in SiteToSiteStatusReportingTask Signed-off-by: Matthew Burgess This closes #4039 --- .../reporting/SiteToSiteStatusReportingTask.java | 62 +- .../additionalDetails.html | 2 + .../src/main/resources/schema-status.avsc | 2 + .../TestSiteToSiteStatusReportingTask.java | 5 ++ 4 files changed, 46 insertions(+), 25 deletions(-) diff --git a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java index 2466827..31009f8 100644 --- a/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java +++ b/nifi-nar-bundles/nifi-site-to-site-reporting-bundle/nifi-site-to-site-reporting-task/src/main/java/org/apache/nifi/reporting/SiteToSiteStatusReportingTask.java @@ -94,6 +94,7 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa private volatile Pattern componentTypeFilter; private volatile Pattern componentNameFilter; +private volatile Map processGroupIDToPath; public SiteToSiteStatusReportingTask() throws IOException { final InputStream schema = getClass().getClassLoader().getResourceAsStream("schema-status.avsc"); @@ -122,6 +123,9 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa componentTypeFilter = Pattern.compile(context.getProperty(COMPONENT_TYPE_FILTER_REGEX).evaluateAttributeExpressions().getValue()); componentNameFilter = Pattern.compile(context.getProperty(COMPONENT_NAME_FILTER_REGEX).evaluateAttributeExpressions().getValue()); +// initialize the map +processGroupIDToPath = new HashMap(); + final ProcessGroupStatus procGroupStatus = context.getEventAccess().getControllerStatus(); final String rootGroupName = procGroupStatus == null ? null : procGroupStatus.getName(); @@ -145,8 +149,8 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa df.setTimeZone(TimeZone.getTimeZone("Z")); final JsonArrayBuilder arrayBuilder = factory.createArrayBuilder(); -serializeProcessGroupStatus(arrayBuilder, factory, procGroupStatus, df, hostname, rootGroupName, -platform, null, new Date(), allowNullValues); +serializeProcessGroupStatus(arrayBuilder, factory, procGroupStatus, df, +hostname, rootGroupName, platform, null, new Date(), allowNullValues); final JsonArray jsonArray = arrayBuilder.build(); @@ -230,22 +234,26 @@ public class SiteToSiteStatusReportingTask extends AbstractSiteToSiteReportingTa *The root process group name * @param platform *The configured platform - * @param parentId - *The parent's component id + * @param parent + *The parent's process group status object * @param currentDate *The current date * @param allowNullValues *Allow null values */ private void serializeProcessGroupStatus(final JsonArrayBuilder arrayBuilder, final JsonBuilderFactory factory, -final ProcessGroupStatus status, final DateFormat df, -final String hostname, final String applicationName, final String platform, final String parentId, final Date currentDate, Boolean allowNullValues) { +final ProcessGroupStatus status, final DateFormat df, final String hostname, final String applicationName, +final String platform, final ProcessGroupStatus parent, final Date currentDate, Boolean allowNullValues) { final JsonObjectBuilder builder = factory.createObjectBuilder(); -final String componentType = (parentId == null) ? "RootProcessGroup" : "ProcessGroup"; +final String componentType = parent == null ? "RootProcessGroup" : "ProcessGroup"; final String componentName = status.getName(); +if(parent == null) { +processGroupIDToPath.put(status.getId(), "NiFi Flow"); +
[nifi] branch master updated: NIFI-6919: Added relationship attribute to DistributeLoad
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 421bfdd NIFI-6919: Added relationship attribute to DistributeLoad 421bfdd is described below commit 421bfdd5fff87e477a91450f8414d7a989bd083f Author: Michael Hogue AuthorDate: Thu Dec 12 21:06:15 2019 -0500 NIFI-6919: Added relationship attribute to DistributeLoad NIFI-6919: Cleaned up docs NIFI-6919: Cleanup NIFI-6919: Cleanup NIFI-6919: added negative unit test NIFI-6919: Removed unnecesary feature flag Updated attribute description Signed-off-by: Matthew Burgess This closes #3939 --- .../nifi/processors/standard/DistributeLoad.java | 42 -- .../processors/standard/TestDistributeLoad.java| 29 ++- 2 files changed, 51 insertions(+), 20 deletions(-) diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java index 8c1f9bd..57c1723 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/DistributeLoad.java @@ -31,14 +31,16 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.commons.lang3.StringUtils; import org.apache.nifi.annotation.behavior.DynamicProperty; +import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.behavior.EventDriven; import org.apache.nifi.annotation.behavior.InputRequirement; +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; import org.apache.nifi.annotation.behavior.SideEffectFree; import org.apache.nifi.annotation.behavior.SupportsBatching; import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable; -import org.apache.nifi.annotation.behavior.InputRequirement.Requirement; +import org.apache.nifi.annotation.behavior.WritesAttribute; +import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; -import org.apache.nifi.annotation.behavior.DynamicRelationship; import org.apache.nifi.annotation.documentation.Tags; import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.components.PropertyDescriptor; @@ -71,6 +73,9 @@ import org.apache.nifi.processor.util.StandardValidators; + "'5' will be receive 10 FlowFiles in each iteration instead of 1.") @DynamicRelationship(name = "A number 1..", description = "FlowFiles are sent to this relationship per the " + "") +@WritesAttributes( +@WritesAttribute(attribute = "distribute.load.relationship", description = "The name of the specific relationship the flow file has been routed through") +) public class DistributeLoad extends AbstractProcessor { public static final String STRATEGY_ROUND_ROBIN = "round robin"; @@ -93,31 +98,26 @@ public class DistributeLoad extends AbstractProcessor { .allowableValues(STRATEGY_ROUND_ROBIN, STRATEGY_NEXT_AVAILABLE, STRATEGY_LOAD_DISTRIBUTION_SERVICE) .defaultValue(STRATEGY_ROUND_ROBIN) .build(); - public static final PropertyDescriptor HOSTNAMES = new PropertyDescriptor.Builder() .name("Hostnames") .description("List of remote servers to distribute across. Each server must be FQDN and use either ',', ';', or [space] as a delimiter") .required(true) -.addValidator(new Validator() { - -@Override -public ValidationResult validate(String subject, String input, ValidationContext context) { -ValidationResult result = new ValidationResult.Builder().subject(subject).valid(true).input(input).explanation("Good FQDNs").build(); -if (null == input) { +.addValidator((subject, input, context) -> { +ValidationResult result = new ValidationResult.Builder().subject(subject).valid(true).input(input).explanation("Good FQDNs").build(); +if (null == input) { +result = new ValidationResult.Builder().subject(subject).input(input).valid(false) +.explanation("Need to specify delimited list of FQDNs").build(); +return result; +} +String[] hostNames = input.split("
[nifi] branch master updated: NIFI-7210 - added PG path in bulletins for S2S Bulletin RT
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 12c8402 NIFI-7210 - added PG path in bulletins for S2S Bulletin RT 12c8402 is described below commit 12c8402ac3e648dd63b959b415cea61099b10864 Author: Pierre Villard AuthorDate: Thu Feb 27 14:59:03 2020 -0800 NIFI-7210 - added PG path in bulletins for S2S Bulletin RT Added group path to BULLETIN table for QueryNiFiReportingTask Signed-off-by: Matthew Burgess This closes #4100 --- .../java/org/apache/nifi/reporting/Bulletin.java | 9 ++ .../org/apache/nifi/reporting/BulletinFactory.java | 15 ++ .../org/apache/nifi/events/BulletinFactory.java| 32 +- .../reporting/SiteToSiteBulletinReportingTask.java | 1 + .../additionalDetails.html | 1 + .../src/main/resources/schema-bulletins.avsc | 1 + .../TestSiteToSiteBulletinReportingTask.java | 5 ++-- .../sql/bulletins/BulletinEnumerator.java | 1 + .../reporting/sql/bulletins/BulletinTable.java | 2 ++ .../additionalDetails.html | 1 + 10 files changed, 64 insertions(+), 4 deletions(-) diff --git a/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java b/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java index 50684d6..3f160aa 100644 --- a/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java +++ b/nifi-api/src/main/java/org/apache/nifi/reporting/Bulletin.java @@ -33,6 +33,7 @@ public abstract class Bulletin implements Comparable { private String groupId; private String groupName; +private String groupPath; private String sourceId; private String sourceName; private ComponentType sourceType; @@ -98,6 +99,14 @@ public abstract class Bulletin implements Comparable { this.groupName = groupName; } +public String getGroupPath() { +return groupPath; +} + +public void setGroupPath(String groupPath) { +this.groupPath = groupPath; +} + public String getSourceId() { return sourceId; } diff --git a/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java b/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java index 0208a2e..ca1ef6c 100644 --- a/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java +++ b/nifi-mock/src/main/java/org/apache/nifi/reporting/BulletinFactory.java @@ -53,4 +53,19 @@ public class BulletinFactory { bulletin.setMessage(message); return bulletin; } + +public static Bulletin createBulletin(final String groupId, final String groupName, final String sourceId, final ComponentType sourceType, +final String sourceName, final String category, final String severity, final String message, final String groupPath) { +final Bulletin bulletin = new MockBulletin(currentId.getAndIncrement()); +bulletin.setGroupId(groupId); +bulletin.setGroupName(groupName); +bulletin.setGroupPath(groupPath); +bulletin.setSourceId(sourceId); +bulletin.setSourceType(sourceType); +bulletin.setSourceName(sourceName); +bulletin.setCategory(category); +bulletin.setLevel(severity); +bulletin.setMessage(message); +return bulletin; +} } diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java index 76728ba..1d9f8cf 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core-api/src/main/java/org/apache/nifi/events/BulletinFactory.java @@ -49,7 +49,22 @@ public final class BulletinFactory { final ProcessGroup group = connectable.getProcessGroup(); final String groupId = connectable.getProcessGroupIdentifier(); final String groupName = group == null ? null : group.getName(); -return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message); +final String groupPath = buildGroupPath(group); +return BulletinFactory.createBulletin(groupId, groupName, connectable.getIdentifier(), type, connectable.getName(), category, severity, message, groupPath); +} + +private static String buildGroupPath(ProcessGroup group) { +if(group == null) { +return null; +} else { +String path = group.getName(); +ProcessGroup
[nifi] branch master updated: NIFI-7087: Use FlowManager.findAllConnections() when available
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 84968e7 NIFI-7087: Use FlowManager.findAllConnections() when available 84968e7 is described below commit 84968e70d293e49888addc3870c6a9cf222103b0 Author: Matthew Burgess AuthorDate: Wed Feb 12 16:27:56 2020 -0500 NIFI-7087: Use FlowManager.findAllConnections() when available Signed-off-by: Matthew Burgess This closes #4026 --- .../java/org/apache/nifi/controller/FlowController.java | 8 .../status/analytics/ConnectionStatusAnalytics.java | 13 - .../apache/nifi/provenance/ComponentIdentifierLookup.java | 11 ++- .../status/analytics/TestConnectionStatusAnalytics.java | 7 +++ 4 files changed, 21 insertions(+), 18 deletions(-) diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java index 72c3416..33df9be 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/FlowController.java @@ -875,7 +875,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node writeLock.lock(); try { // get all connections/queues and recover from swap files. -final List connections = flowManager.getRootGroup().findAllConnections(); +final Set connections = flowManager.findAllConnections(); flowFileRepository.loadFlowFiles(queueProvider); @@ -1086,7 +1086,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node startRemoteGroupPortsAfterInitialization.clear(); } -for (final Connection connection : flowManager.getRootGroup().findAllConnections()) { +for (final Connection connection : flowManager.findAllConnections()) { connection.getFlowFileQueue().startLoadBalancing(); } } finally { @@ -2645,7 +2645,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node return "Cannot replay data from Provenance Event because the event does not specify the Source FlowFile Queue"; } -final List connections = flowManager.getRootGroup().findAllConnections(); +final Set connections = flowManager.findAllConnections(); FlowFileQueue queue = null; for (final Connection connection : connections) { if (event.getSourceQueueIdentifier().equals(connection.getIdentifier())) { @@ -2696,7 +2696,7 @@ public class FlowController implements ReportingTaskProvider, Authorizable, Node throw new IllegalArgumentException("Cannot replay data from Provenance Event because the event does not specify the Source FlowFile Queue"); } -final List connections = flowManager.getRootGroup().findAllConnections(); +final Set connections = flowManager.findAllConnections(); FlowFileQueue queue = null; for (final Connection connection : connections) { if (event.getSourceQueueIdentifier().equals(connection.getIdentifier())) { diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java index a524566..5500fbb 100644 --- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java +++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/main/java/org/apache/nifi/controller/status/analytics/ConnectionStatusAnalytics.java @@ -21,7 +21,6 @@ import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.NoSuchElementException; -import java.util.Optional; import java.util.stream.Stream; import org.apache.commons.collections4.MapUtils; @@ -33,7 +32,6 @@ import org.apache.nifi.controller.repository.FlowFileEventRepository; import org.apache.nifi.controller.repository.RepositoryStatusReport; import org.apache.nifi.controller.status.history.ComponentStatusRepository; import org.apache.nifi.controller.status.history.StatusHistory; -import org.apache.nifi.groups.P
[nifi] branch master updated: NIFI-7359 Fix parent id on process metrics for Prometheus
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new b113a02 NIFI-7359 Fix parent id on process metrics for Prometheus b113a02 is described below commit b113a022e46ee755e6b9d4395b908c7f54b2bd37 Author: Alexandre Vautier AuthorDate: Tue Apr 14 22:21:52 2020 +0200 NIFI-7359 Fix parent id on process metrics for Prometheus Signed-off-by: Matthew Burgess This closes #4209 --- .../prometheus/util/PrometheusMetricsUtil.java | 24 +++--- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/src/main/java/org/apache/nifi/prometheus/util/PrometheusMetricsUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/src/main/java/org/apache/nifi/prometheus/util/PrometheusMetricsUtil.java index 0ac5f47..01b91a8 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/src/main/java/org/apache/nifi/prometheus/util/PrometheusMetricsUtil.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-prometheus-utils/src/main/java/org/apache/nifi/prometheus/util/PrometheusMetricsUtil.java @@ -438,28 +438,28 @@ public class PrometheusMetricsUtil { final String procComponentName = processorStatus.getName(); final String parentId = processorStatus.getGroupId(); -AMOUNT_FLOWFILES_SENT.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getFlowFilesSent()); -AMOUNT_FLOWFILES_RECEIVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getFlowFilesReceived()); -AMOUNT_FLOWFILES_REMOVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getFlowFilesRemoved()); +AMOUNT_FLOWFILES_SENT.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getFlowFilesSent()); +AMOUNT_FLOWFILES_RECEIVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getFlowFilesReceived()); +AMOUNT_FLOWFILES_REMOVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getFlowFilesRemoved()); -AMOUNT_BYTES_SENT.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getBytesSent()); -AMOUNT_BYTES_READ.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getBytesRead()); -AMOUNT_BYTES_WRITTEN.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getBytesWritten()); +AMOUNT_BYTES_SENT.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getBytesSent()); +AMOUNT_BYTES_READ.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getBytesRead()); +AMOUNT_BYTES_WRITTEN.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getBytesWritten()); TOTAL_BYTES_READ.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).inc(status.getBytesRead()); TOTAL_BYTES_WRITTEN.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).inc(status.getBytesWritten()); -AMOUNT_BYTES_RECEIVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId).set(processorStatus.getBytesReceived()); +AMOUNT_BYTES_RECEIVED.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId).set(processorStatus.getBytesReceived()); -SIZE_CONTENT_OUTPUT_TOTAL.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId, "", "", "", "") +SIZE_CONTENT_OUTPUT_TOTAL.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId, "", "", "", "") .set(processorStatus.getOutputBytes()); -SIZE_CONTENT_INPUT_TOTAL.labels(instanceId, procComponentType, procComponentName, procComponentId, parentPGId, "", "", "", "") +SIZE_CONTENT_INPUT_TOTAL.labels(instanceId, procComponentType, procComponentName, procComponentId, parentId, "", "", "", "")
[nifi] branch master updated: NIFI-7287: Move services-api dependency from Prometheus reporting task to its NAR
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new be2929b NIFI-7287: Move services-api dependency from Prometheus reporting task to its NAR be2929b is described below commit be2929b8b02859bc1ad0b779d3234547297052f2 Author: Matthew Burgess AuthorDate: Thu Mar 26 10:23:11 2020 -0400 NIFI-7287: Move services-api dependency from Prometheus reporting task to its NAR Signed-off-by: Matthew Burgess This closes #4162 --- nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/pom.xml | 6 ++ .../nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml | 6 -- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/pom.xml b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/pom.xml index f7e0c63..6cbe4ca 100644 --- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/pom.xml +++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-nar/pom.xml @@ -28,6 +28,12 @@ org.apache.nifi +nifi-standard-services-api-nar +1.12.0-SNAPSHOT +nar + + +org.apache.nifi nifi-prometheus-reporting-task 1.12.0-SNAPSHOT diff --git a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml index f2d3b3c..300c601 100644 --- a/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml +++ b/nifi-nar-bundles/nifi-prometheus-bundle/nifi-prometheus-reporting-task/pom.xml @@ -88,12 +88,6 @@ 1.12.0-SNAPSHOT -org.apache.nifi -nifi-standard-services-api-nar -1.12.0-SNAPSHOT -nar - - org.eclipse.jetty jetty-continuation ${jetty.version}
[nifi] branch master updated (daddf40 -> 4a2a911)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from daddf40 NIFI-7103 Adding PutDataLakeStorage Processor to provide native support for Azure Data Lake Storage Gen 2 Storage. add 4a2a911 NIFI-7293 Add in-memory janusgraph implementation of GraphClientService to help with live testing. No new revisions were added by this update. Summary of changes: .../nifi-graph-processors/pom.xml | 6 ++ .../nifi/processors/graph/ExecuteGraphQueryIT.java | 65 ++ .../nifi-graph-test-clients}/pom.xml | 36 +--- .../graph/InMemoryJanusGraphClientService.java | 100 + .../InMemoryJanusGraphClientServiceTest.groovy | 63 + nifi-nar-bundles/nifi-graph-bundle/pom.xml | 1 + 6 files changed, 257 insertions(+), 14 deletions(-) create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-processors/src/test/java/org/apache/nifi/processors/graph/ExecuteGraphQueryIT.java copy nifi-nar-bundles/{nifi-splunk-bundle/nifi-splunk-processors => nifi-graph-bundle/nifi-graph-test-clients}/pom.xml (74%) create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/src/main/java/org/apache/nifi/graph/InMemoryJanusGraphClientService.java create mode 100644 nifi-nar-bundles/nifi-graph-bundle/nifi-graph-test-clients/src/test/groovy/org/apache/nifi/graph/InMemoryJanusGraphClientServiceTest.groovy
[nifi] branch master updated (659a383 -> 7784178)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from 659a383 NIFI-7394: Add support for sending Multipart/FORM data to InvokeHTTP. By using dynamic properties with a prefix naming scheme, allow definition of the parts, including the name to give the Flowfile content part, and optionally it's file name. After review: - change so that we can send just the form content or just form data without the flowfile - change the content name and content file name from dynamic properties to properties - change the dynamic name to be an inv [...] add 7784178 NIFI-7408 - added percent used metrics for connections No new revisions were added by this update. Summary of changes: .../apache/nifi/prometheus/util/NiFiMetricsRegistry.java | 14 ++ .../apache/nifi/prometheus/util/PrometheusMetricsUtil.java | 10 ++ 2 files changed, 24 insertions(+)
[nifi] branch master updated (c51b905 -> 13418cc)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git. from c51b905 NIFI-7460: Avoid NPE when a VersionedProcessor has a null value for autoTerminatedRelationships. Added additional logging and improved error handling around syncing with invalid flows add 13418cc NIFI-7437 - created separate thread for preloading predictions, refactors for performance No new revisions were added by this update. Summary of changes: .../org/apache/nifi/controller/FlowController.java | 24 - .../CachingConnectionStatusAnalyticsEngine.java| 10 +- .../analytics/ConnectionStatusAnalytics.java | 111 ++--- .../analytics/ConnectionStatusAnalyticsEngine.java | 8 +- ...TestCachingConnectionStatusAnalyticsEngine.java | 7 +- .../analytics/TestConnectionStatusAnalytics.java | 68 ++--- .../TestConnectionStatusAnalyticsEngine.java | 9 +- .../analytics/TestStatusAnalyticsEngine.java | 4 +- 8 files changed, 120 insertions(+), 121 deletions(-)
[nifi] branch master updated: NIFI-7390 Covering Avro type conversion in case of map withing Record
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 66b175f NIFI-7390 Covering Avro type conversion in case of map withing Record 66b175f is described below commit 66b175f405e727aba650f67872292ff8c8ccb14d Author: Bence Simon AuthorDate: Wed May 6 16:04:36 2020 +0200 NIFI-7390 Covering Avro type conversion in case of map withing Record Signed-off-by: Matthew Burgess This closes #4256 --- .../java/org/apache/nifi/avro/AvroTypeUtil.java| 2 +- .../org/apache/nifi/avro/TestAvroTypeUtil.java | 51 ++ 2 files changed, 52 insertions(+), 1 deletion(-) diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java index caa2743..950e4cb 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/main/java/org/apache/nifi/avro/AvroTypeUtil.java @@ -747,7 +747,7 @@ public class AvroTypeUtil { for (final RecordField recordField : recordValue.getSchema().getFields()) { final Object v = recordValue.getValue(recordField); if (v != null) { -map.put(recordField.getFieldName(), v); +map.put(recordField.getFieldName(), convertToAvroObject(v, fieldSchema.getValueType(), fieldName + "[" + recordField.getFieldName() + "]", charset)); } } diff --git a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java index 5b5f55dd..412d573 100644 --- a/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java +++ b/nifi-nar-bundles/nifi-extension-utils/nifi-record-utils/nifi-avro-record-utils/src/test/java/org/apache/nifi/avro/TestAvroTypeUtil.java @@ -33,6 +33,7 @@ import org.apache.avro.util.Utf8; 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.MapRecord; import org.apache.nifi.serialization.record.RecordField; import org.apache.nifi.serialization.record.RecordFieldType; import org.apache.nifi.serialization.record.RecordSchema; @@ -691,4 +692,54 @@ public class TestAvroTypeUtil { // THEN assertEquals(expected, actual); } + +@Test +public void testConvertNifiRecordIntoAvroRecord() throws IOException { +// given +final MapRecord nifiRecord = givenRecordContainingNumericMap(); +final Schema avroSchema = givenAvroSchemaContainingNumericMap(); + +// when +final GenericRecord result = AvroTypeUtil.createAvroRecord(nifiRecord, avroSchema); + +// then +final HashMap numbers = (HashMap) result.get("numbers"); +Assert.assertTrue(Long.class.isInstance(numbers.get("number1"))); +Assert.assertTrue(Long.class.isInstance(numbers.get("number2"))); +} + +private MapRecord givenRecordContainingNumericMap() { + +final Map numberValues = new HashMap<>(); +numberValues.put("number1", 123); // Intentionally an Integer as validation accepts it +numberValues.put("number2", 123L); + +final List numberFields = Arrays.asList( +new RecordField("number1", RecordFieldType.LONG.getDataType()), +new RecordField("number2", RecordFieldType.LONG.getDataType()) +); + +final RecordSchema nifiNumberSchema = new SimpleRecordSchema(numberFields); +final MapRecord numberRecord = new MapRecord(new SimpleRecordSchema(numberFields), numberValues); + +final Map values = new HashMap<>(); +values.put("id", 1); +values.put("numbers", numberRecord); + +final List fields = Arrays.asList( +new RecordField("id", RecordFieldType.INT.getDataType()), +new RecordField("numbers", RecordFieldType.RECORD.getRecordDataType(nifiNumberSchema)) +); + +return new MapRecord(new SimpleReco
[nifi] branch master updated: NIFI-7413: Documented REMOTE_INVOCATION provenance event type in user/dev guides
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new 6b73ea4 NIFI-7413: Documented REMOTE_INVOCATION provenance event type in user/dev guides 6b73ea4 is described below commit 6b73ea48bc4d2b80e69ba37efa3a4c6ed8f80e6a Author: Peter Gyori AuthorDate: Tue May 12 10:22:43 2020 +0200 NIFI-7413: Documented REMOTE_INVOCATION provenance event type in user/dev guides Signed-off-by: Matthew Burgess This closes #4267 --- nifi-docs/src/main/asciidoc/developer-guide.adoc | 1 + nifi-docs/src/main/asciidoc/user-guide.adoc | 1 + 2 files changed, 2 insertions(+) diff --git a/nifi-docs/src/main/asciidoc/developer-guide.adoc b/nifi-docs/src/main/asciidoc/developer-guide.adoc index aca8b69..2c84b77 100644 --- a/nifi-docs/src/main/asciidoc/developer-guide.adoc +++ b/nifi-docs/src/main/asciidoc/developer-guide.adoc @@ -900,6 +900,7 @@ The different event types for provenance reporting are: |FORK|Indicates that one or more FlowFiles were derived from a parent FlowFile |JOIN|Indicates that a single FlowFile is derived from joining together multiple parent FlowFiles |RECEIVE |Indicates a provenance event for receiving data from an external process. This Event Type is expected to be the first event for a FlowFile. As such, a Processor that receives data from an external source and uses that data to replace the content of an existing FlowFile should use the FETCH event type, rather than the RECEIVE event type +|REMOTE_INVOCATION |Indicates that a remote invocation was requested to an external endpoint (e.g. deleting a remote resource). The external endpoint may exist in a remote or a local system, but is external to NiFi |REPLAY |Indicates a provenance event for replaying a FlowFile. The UUID of the event indicates the UUID of the original FlowFile that is being replayed. The event contains one Parent UUID that is also the UUID of the FlowFile that is being replayed and one Child UUID that is the UUID of the a newly created FlowFile that will be re-queued for processing |ROUTE |Indicates that a FlowFile was routed to a specified relationship and provides information about why the FlowFile was routed to this relationship |SEND|Indicates a provenance event for sending data to an external process diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc index 10a1c22..c965f1e 100644 --- a/nifi-docs/src/main/asciidoc/user-guide.adoc +++ b/nifi-docs/src/main/asciidoc/user-guide.adoc @@ -2594,6 +2594,7 @@ The provenance event types are: |FORK|Indicates that one or more FlowFiles were derived from a parent FlowFile |JOIN|Indicates that a single FlowFile is derived from joining together multiple parent FlowFiles |RECEIVE |Indicates a provenance event for receiving data from an external process +|REMOTE_INVOCATION |Indicates that a remote invocation was requested to an external endpoint (e.g. deleting a remote resource) |REPLAY |Indicates a provenance event for replaying a FlowFile |ROUTE |Indicates that a FlowFile was routed to a specified relationship and provides information about why the FlowFile was routed to this relationship |SEND|Indicates a provenance event for sending data to an external process
[nifi-registry] branch master updated: NIFIREG-390: Add .asf.yaml file to GitHub repo
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi-registry.git The following commit(s) were added to refs/heads/master by this push: new df00b46 NIFIREG-390: Add .asf.yaml file to GitHub repo df00b46 is described below commit df00b461782c3e5805aead1b282a58d5aa0cfd3b Author: Matthew Burgess AuthorDate: Tue May 19 15:23:52 2020 -0400 NIFIREG-390: Add .asf.yaml file to GitHub repo Signed-off-by: Matthew Burgess This closes #280 --- .asf.yaml | 34 ++ 1 file changed, 34 insertions(+) diff --git a/.asf.yaml b/.asf.yaml new file mode 100644 index 000..fc379fc --- /dev/null +++ b/.asf.yaml @@ -0,0 +1,34 @@ +# 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. +github: + description: "Apache NiFi Registry" + homepage: https://nifi.apache.org/ + labels: +- nifi +- java + features: +wiki: false +issues: false +projects: false + enabled_merge_buttons: +squash: true +merge: true +rebase: true +notifications: +commits: commits@nifi.apache.org +issues: iss...@nifi.apache.org +pullrequests: iss...@nifi.apache.org +jira_options: link label worklog \ No newline at end of file
[nifi] branch MINIFI-422 created (now b6ef7e1)
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a change to branch MINIFI-422 in repository https://gitbox.apache.org/repos/asf/nifi.git. at b6ef7e1 NIFI-7462: This adds a way to convert or cast a choice object into a valid type for use with calcite query functions No new revisions were added by this update.
[nifi] branch master updated: NIFI-7462: This adds a way to convert or cast a choice object into a valid type for use with calcite query functions
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch master in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/master by this push: new b6ef7e1 NIFI-7462: This adds a way to convert or cast a choice object into a valid type for use with calcite query functions b6ef7e1 is described below commit b6ef7e13bf076fb88fd94ce49d2a217db3f19aaa Author: pcgrenier AuthorDate: Fri May 15 20:03:01 2020 -0400 NIFI-7462: This adds a way to convert or cast a choice object into a valid type for use with calcite query functions NIFI-7462: Update to allow FlowFile Table's schema to be more intelligent when using CHOICE types NIFI-7462: Fixed checkstyle violation, removed documentation around the CAST functions that were no longer needed Signed-off-by: Matthew Burgess This closes #4282 --- .../nifi/processors/standard/QueryRecord.java | 26 ++--- .../org/apache/nifi/queryrecord/FlowFileTable.java | 58 + .../nifi/processors/standard/TestQueryRecord.java | 129 + 3 files changed, 199 insertions(+), 14 deletions(-) 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 82aea6f..a620a60 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 @@ -784,12 +784,6 @@ public class QueryRecord extends AbstractProcessor { } } -public static class RecordRecordPath extends RecordPathFunction { -public Record eval(Object record, String recordPath) { -return eval(record, recordPath, Record.class::cast); -} -} - public static class RecordPathFunction { private static final RecordField ROOT_RECORD_FIELD = new RecordField("root", RecordFieldType.MAP.getMapDataType(RecordFieldType.STRING.getDataType())); @@ -803,14 +797,18 @@ public class QueryRecord extends AbstractProcessor { return null; } -if (record instanceof Record) { -return eval((Record) record, recordPath, transform); -} else if (record instanceof Record[]) { -return eval((Record[]) record, recordPath, transform); -} else if (record instanceof Iterable) { -return eval((Iterable) record, recordPath, transform); -} else if (record instanceof Map) { -return eval((Map) record, recordPath, transform); +try { +if (record instanceof Record) { +return eval((Record) record, recordPath, transform); +} else if (record instanceof Record[]) { +return eval((Record[]) record, recordPath, transform); +} else if (record instanceof Iterable) { +return eval((Iterable) record, recordPath, transform); +} else if (record instanceof Map) { +return eval((Map) record, recordPath, transform); +} +} catch (IllegalArgumentException e) { +throw new RuntimeException("Cannot evaluate RecordPath " + recordPath + " against " + record, e); } throw new RuntimeException("Cannot evaluate RecordPath " + recordPath + " against given argument because the argument is of type " + record.getClass() + " instead of Record"); diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java index 3030008..18cbc63 100644 --- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java +++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/queryrecord/FlowFileTable.java @@ -43,6 +43,7 @@ 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.type.ArrayDataType; +import org.apache.nifi.serialization.record.type.ChoiceDataType; import java.lang.reflect.Type; import java.math.BigInteger; @@ -223,12 +224,69 @@ public class FlowFileTable extends AbstractTable implements QueryableTable, Tran case BIGINT:
[nifi] branch main updated: NIFI-7743 Document Empty all queues option for Process Groups
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/main by this push: new 29e23e5 NIFI-7743 Document Empty all queues option for Process Groups 29e23e5 is described below commit 29e23e57b909f0deed2929249792ba04d2b6f590 Author: Andrew Lim AuthorDate: Tue Sep 1 12:56:33 2020 -0400 NIFI-7743 Document Empty all queues option for Process Groups Signed-off-by: Matthew Burgess This closes #4506 --- .../asciidoc/images/configure-process-group.png| Bin 73011 -> 38116 bytes .../asciidoc/images/nifi-process-group-menu.png| Bin 95017 -> 120094 bytes .../images/process-group-configuration-window.png | Bin 75251 -> 102300 bytes nifi-docs/src/main/asciidoc/user-guide.adoc| 80 - 4 files changed, 45 insertions(+), 35 deletions(-) diff --git a/nifi-docs/src/main/asciidoc/images/configure-process-group.png b/nifi-docs/src/main/asciidoc/images/configure-process-group.png index a6a4d41..aeb54de 100644 Binary files a/nifi-docs/src/main/asciidoc/images/configure-process-group.png and b/nifi-docs/src/main/asciidoc/images/configure-process-group.png differ diff --git a/nifi-docs/src/main/asciidoc/images/nifi-process-group-menu.png b/nifi-docs/src/main/asciidoc/images/nifi-process-group-menu.png index c7affa3..d8e0ea7 100644 Binary files a/nifi-docs/src/main/asciidoc/images/nifi-process-group-menu.png and b/nifi-docs/src/main/asciidoc/images/nifi-process-group-menu.png differ diff --git a/nifi-docs/src/main/asciidoc/images/process-group-configuration-window.png b/nifi-docs/src/main/asciidoc/images/process-group-configuration-window.png index 7566010..8921129 100644 Binary files a/nifi-docs/src/main/asciidoc/images/process-group-configuration-window.png and b/nifi-docs/src/main/asciidoc/images/process-group-configuration-window.png differ diff --git a/nifi-docs/src/main/asciidoc/user-guide.adoc b/nifi-docs/src/main/asciidoc/user-guide.adoc index 4894c63..cd2fc65 100644 --- a/nifi-docs/src/main/asciidoc/user-guide.adoc +++ b/nifi-docs/src/main/asciidoc/user-guide.adoc @@ -373,6 +373,7 @@ NOTE: It is also possible to double-click on the Process Group to enter it. - *Download flow*: This option allows the user to download the flow as a JSON file. The file can be used as a backup or imported into a link:https://nifi.apache.org/registry.html[NiFi Registry^] using the <>. (Note: If "Download flow" is selected for a versioned process group, there is no versioning information in the download. In other words, the resulting contents of the JSON file is the same whether the process group is versioned or not.) - *Create template*: This option allows the user to create a template from the selected Process Group. - *Copy*: This option places a copy of the selected Process Group on the clipboard, so that it may be pasted elsewhere on the canvas by right-clicking on the canvas and selecting Paste. The Copy/Paste actions also may be done using the keystrokes Ctrl-C (Command-C) and Ctrl-V (Command-V). +- *Empty all queues*: This option allows the user to empty all queues in the selected Process Group. All FlowFiles from all connections waiting at the time of the request will be removed. - *Delete*: This option allows the DFM to delete a Process Group. @@ -726,31 +727,35 @@ You can access additional documentation about each Processor's usage by right-cl [[Configuring_a_ProcessGroup]] === Configuring a Process Group -To configure a Process Group, right-click on the Process Group and select the `Configure` option from the context menu. -This will provide a configuration dialog such as the dialog below: +To configure a Process Group, right-click on the Process Group and select the `Configure` option from the context menu. The configuration dialog is opened with two tabs: General and Controller Services. image::configure-process-group.png["Configure Process Group"] -Process Groups provide a few different configuration options. First is the name of the Process Group. This is the name that is -shown at the top of the Process Group on the canvas as well as in the breadcrumbs at the bottom of the UI. For the Root Process -Group (i.e., the highest level group), this is also the name that is shown as the title of the browser tab. -The next configuration element is the <>, which is used to provide parameters to components of the flow. -From this screen, the user is able to choose which Parameter Context should be bound to this Process Group and can optionally -create a new one to bind to the Process Group. Parameters and Parameter Contexts are covered in detail in the next section. +[[General_tab_ProcessGroup]] + General Tab +This tab contains several different configuration items. First is the Process Group Name.
[nifi] branch main updated: NIFI-7800: Provide an option to omit XML declaration for XMLRecordSetWriter
This is an automated email from the ASF dual-hosted git repository. mattyb149 pushed a commit to branch main in repository https://gitbox.apache.org/repos/asf/nifi.git The following commit(s) were added to refs/heads/main by this push: new fd8b0b2 NIFI-7800: Provide an option to omit XML declaration for XMLRecordSetWriter fd8b0b2 is described below commit fd8b0b286fbb83cfdba3a24b2a3e6f647cf395bf Author: Mohammed Nadeem AuthorDate: Fri Sep 11 03:32:50 2020 +0530 NIFI-7800: Provide an option to omit XML declaration for XMLRecordSetWriter NIFI-7800: Mark new property as required Signed-off-by: Matthew Burgess This closes #4520 --- .../java/org/apache/nifi/xml/WriteXMLResult.java | 9 +- .../org/apache/nifi/xml/XMLRecordSetWriter.java| 14 +- .../org/apache/nifi/xml/TestWriteXMLResult.java| 150 + 3 files changed, 111 insertions(+), 62 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/xml/WriteXMLResult.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java index 8b4710b..6f5abc8 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/WriteXMLResult.java @@ -60,6 +60,7 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet private final SchemaAccessWriter schemaAccess; private final XMLStreamWriter writer; private final NullSuppression nullSuppression; +private final boolean omitDeclaration; private final ArrayWrapping arrayWrapping; private final String arrayTagName; private final String recordTagName; @@ -71,7 +72,7 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet private final Supplier LAZY_TIME_FORMAT; private final Supplier LAZY_TIMESTAMP_FORMAT; -public WriteXMLResult(final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final OutputStream out, final boolean prettyPrint, +public WriteXMLResult(final RecordSchema recordSchema, final SchemaAccessWriter schemaAccess, final OutputStream out, final boolean prettyPrint, final boolean omitDeclaration, final NullSuppression nullSuppression, final ArrayWrapping arrayWrapping, final String arrayTagName, final String rootTagName, final String recordTagName, final String charSet, final String dateFormat, final String timeFormat, final String timestampFormat) throws IOException { @@ -81,6 +82,8 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet this.schemaAccess = schemaAccess; this.nullSuppression = nullSuppression; +this.omitDeclaration = omitDeclaration; + this.arrayWrapping = arrayWrapping; this.arrayTagName = arrayTagName; @@ -131,7 +134,9 @@ public class WriteXMLResult extends AbstractRecordSetWriter implements RecordSet schemaAccess.writeHeader(recordSchema, out); try { -writer.writeStartDocument(); +if (!omitDeclaration) { +writer.writeStartDocument(); +} if (allowWritingMultipleRecords) { writer.writeStartElement(rootTagName); diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java index 6ad7006..bef9dec 100644 --- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java +++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordSetWriter.java @@ -80,6 +80,16 @@ public class XMLRecordSetWriter extends DateTimeTextRecordSetWriter implements R .required(true) .build(); +public static final PropertyDescriptor OMIT_XML_DECLARATION = new PropertyDescriptor.Builder() +.name("omit_xml_declaration") +.displayName("Omit XML Declaration") +.description("Specifies whether or not to include XML declaration") +.expressionLanguageSupporte