You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@nifi.apache.org by jo...@apache.org on 2022/05/09 17:58:42 UTC

[nifi] branch support/nifi-1.16 updated (c95d8a95ef -> 9d0e92eb34)

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

joewitt pushed a change to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git


    from c95d8a95ef Merge branch 'NIFI-9868-RC1' into support/nifi-1.16
     new 45ba416d2a NIFI-9918: 'Parse XML Attributes' property added to XMLReader NIFI-9918: Fixed review findings NIFI-9918: Example added to XMLReader's additionalDetails.html on the new property. Minor documentation fixes.
     new 008ecfafb5 NIFI-9944: Fixed issue with using modules in InvokeScriptedProcessor
     new 028982ce0b NIFI-9944 Configured maven-dependency-plugin for JavaScript test JAR
     new f98d7299aa NIFI-9968 Added null check before System.setProperty() in test methods
     new e80e099265 NIFI-9976 Upgraded json-smart to 2.4.8
     new 25031dd819 NIFI-9977 In StandardOauth2AccessTokenProvider add new property to be able to set "scope".
     new 6596eb8f8e NIFI-9798 Added Proxy Support for ConsumeGCPubSub and PublishGCPubSub
     new b114ee0f46 NIFI-9978: - Updating the condition under which we show the Download Flow menu items.
     new e3b85d4b14 NIFI-8533: Replace deprecated jython-shaded with jython-slim
     new 0f6d7c1e7d NIFI-9980 Corrected conflicting GCP dependencies
     new 010e0bc75d NIFI-9932 Upgrade minimist from 1.2.5 to 1.2.6
     new 409566c97d NIFI-9988 Corrected Property Decryption for Authorizers and Providers
     new 237b0abbf3 NIFI-9984 Allow 200-series responses in OAuth2 Access Token Provider
     new a213ad6c61 NIFI-9993: Fixed bug in initialization in which the Content Repo did not properly increment the counter for how many files exist in the archive directories. This was causing the counter to become negative in some cases, which caused processors to incorrectly pause, waiting for content archive cleanup to occur when, in fact, there were no files archived
     new 51610078b2 NIFI-9998: This closes #6023. Upgrade Hive3 to 3.1.3
     new 9d0e92eb34 NIFI-10005

The 16 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:
 .../scheme/StandardProtectionSchemeResolver.java   |   4 +-
 .../StandardProtectionSchemeResolverTest.java      |   9 +
 .../apache/nifi/record/path/TestRecordPath.java    |   4 +-
 nifi-docker/dockerhub/DockerImage.txt              |   2 +-
 nifi-docker/dockerhub/Dockerfile                   |   2 +-
 .../repository/FileSystemRepository.java           |  35 ++-
 .../repository/TestFileSystemRepository.java       |  13 ++
 .../main/webapp/js/nf/canvas/nf-context-menu.js    |   4 +-
 nifi-nar-bundles/nifi-framework-bundle/pom.xml     |   5 -
 .../nifi-gcp-bundle/nifi-gcp-processors/pom.xml    |   2 -
 .../pubsub/AbstractGCPubSubWithProxyProcessor.java |  85 +++++++
 .../processors/gcp/pubsub/ConsumeGCPubSub.java     |  14 +-
 .../processors/gcp/pubsub/PublishGCPubSub.java     |  16 +-
 .../nifi-gcp-bundle/nifi-gcp-services-api/pom.xml  |  12 -
 nifi-nar-bundles/nifi-gcp-bundle/pom.xml           |  12 +-
 .../nifi-hive-bundle/nifi-hive3-processors/pom.xml |  77 +------
 nifi-nar-bundles/nifi-hive-bundle/pom.xml          |   2 +-
 .../nifi-scripting-processors/pom.xml              |  32 ++-
 .../processors/script/InvokeScriptedProcessor.java |   4 -
 .../apache/nifi/script/ScriptRunnerFactory.java    |   2 +-
 .../processors/script/TestInvokeJavascript.java    |  64 +++---
 .../src/test/resources/javascript/test_reader.js   |   4 +-
 .../processors/standard/TestConvertRecord.java     |   4 +-
 .../oauth2/StandardOauth2AccessTokenProvider.java  |  36 ++-
 .../StandardOauth2AccessTokenProviderTest.java     |  15 +-
 .../main/java/org/apache/nifi/xml/XMLReader.java   |  21 +-
 .../java/org/apache/nifi/xml/XMLRecordReader.java  | 120 +++++-----
 .../apache/nifi/xml/inference/XmlRecordSource.java |  20 +-
 .../additionalDetails.html                         | 196 +++++++++++-----
 .../org/apache/nifi/xml/TestInferXmlSchema.java    |  25 ++-
 .../java/org/apache/nifi/xml/TestXMLReader.java    |  21 +-
 .../org/apache/nifi/xml/TestXMLRecordReader.java   | 247 +++++++++++++++------
 .../src/main/package-lock.json                     | 116 +---------
 pom.xml                                            |   6 +
 34 files changed, 747 insertions(+), 484 deletions(-)
 create mode 100644 nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/AbstractGCPubSubWithProxyProcessor.java


[nifi] 07/16: NIFI-9798 Added Proxy Support for ConsumeGCPubSub and PublishGCPubSub

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 6596eb8f8e1e654c6698150ca527a65d65746fe7
Author: Juldrixx <ju...@gmail.com>
AuthorDate: Mon Mar 14 18:24:33 2022 -0500

    NIFI-9798 Added Proxy Support for ConsumeGCPubSub and PublishGCPubSub
    
    This closes #5868
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../pubsub/AbstractGCPubSubWithProxyProcessor.java | 85 ++++++++++++++++++++++
 .../processors/gcp/pubsub/ConsumeGCPubSub.java     | 14 ++--
 .../processors/gcp/pubsub/PublishGCPubSub.java     | 16 ++--
 3 files changed, 101 insertions(+), 14 deletions(-)

diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/AbstractGCPubSubWithProxyProcessor.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/AbstractGCPubSubWithProxyProcessor.java
new file mode 100644
index 0000000000..c4d3e68b68
--- /dev/null
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/AbstractGCPubSubWithProxyProcessor.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.gcp.pubsub;
+
+import com.google.api.gax.rpc.TransportChannelProvider;
+import com.google.cloud.pubsub.v1.TopicAdminSettings;
+import io.grpc.HttpConnectProxiedSocketAddress;
+import io.grpc.ProxiedSocketAddress;
+import io.grpc.ProxyDetector;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processors.gcp.ProxyAwareTransportFactory;
+import org.apache.nifi.proxy.ProxyConfiguration;
+
+import javax.annotation.Nullable;
+import java.net.InetSocketAddress;
+import java.net.Proxy;
+import java.net.SocketAddress;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+
+public abstract class AbstractGCPubSubWithProxyProcessor extends AbstractGCPubSubProcessor {
+    @Override
+    public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return Collections.unmodifiableList(Arrays.asList(
+                PROJECT_ID,
+                ProxyConfiguration.createProxyConfigPropertyDescriptor(true, ProxyAwareTransportFactory.PROXY_SPECS),
+                GCP_CREDENTIALS_PROVIDER_SERVICE)
+        );
+    }
+    protected TransportChannelProvider getTransportChannelProvider(ProcessContext context) {
+        final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context, () -> {
+            final String proxyHost = context.getProperty(PROXY_HOST).evaluateAttributeExpressions().getValue();
+            final Integer proxyPort = context.getProperty(PROXY_PORT).evaluateAttributeExpressions().asInteger();
+            if (proxyHost != null && proxyPort != null && proxyPort > 0) {
+                final ProxyConfiguration componentProxyConfig = new ProxyConfiguration();
+                final String proxyUser = context.getProperty(HTTP_PROXY_USERNAME).evaluateAttributeExpressions().getValue();
+                final String proxyPassword = context.getProperty(HTTP_PROXY_PASSWORD).evaluateAttributeExpressions().getValue();
+                componentProxyConfig.setProxyType(Proxy.Type.HTTP);
+                componentProxyConfig.setProxyServerHost(proxyHost);
+                componentProxyConfig.setProxyServerPort(proxyPort);
+                componentProxyConfig.setProxyUserName(proxyUser);
+                componentProxyConfig.setProxyUserPassword(proxyPassword);
+                return componentProxyConfig;
+            }
+            return ProxyConfiguration.DIRECT_CONFIGURATION;
+        });
+
+        return TopicAdminSettings.defaultGrpcTransportProviderBuilder()
+                .setChannelConfigurator(managedChannelBuilder -> managedChannelBuilder.proxyDetector(
+                        new ProxyDetector() {
+                            @Nullable
+                            @Override
+                            public ProxiedSocketAddress proxyFor(SocketAddress socketAddress) {
+                                if (Proxy.Type.HTTP.equals(proxyConfiguration.getProxyType())) {
+                                    return HttpConnectProxiedSocketAddress.newBuilder()
+                                            .setUsername(proxyConfiguration.getProxyUserName())
+                                            .setPassword(proxyConfiguration.getProxyUserPassword())
+                                            .setProxyAddress(new InetSocketAddress(proxyConfiguration.getProxyServerHost(),
+                                                    proxyConfiguration.getProxyServerPort()))
+                                            .setTargetAddress((InetSocketAddress) socketAddress)
+                                            .build();
+                                } else {
+                                    return null;
+                                }
+                            }
+                        }))
+                .build();
+    }
+}
\ No newline at end of file
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/ConsumeGCPubSub.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/ConsumeGCPubSub.java
index c2cce82a9e..3928a2ae63 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/ConsumeGCPubSub.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/ConsumeGCPubSub.java
@@ -50,7 +50,6 @@ import org.apache.nifi.processor.util.StandardValidators;
 
 import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
@@ -83,7 +82,7 @@ import static org.apache.nifi.processors.gcp.pubsub.PubSubAttributes.SERIALIZED_
         @WritesAttribute(attribute = MSG_PUBLISH_TIME_ATTRIBUTE, description = MSG_PUBLISH_TIME_DESCRIPTION),
         @WritesAttribute(attribute = DYNAMIC_ATTRIBUTES_ATTRIBUTE, description = DYNAMIC_ATTRIBUTES_DESCRIPTION)
 })
-public class ConsumeGCPubSub extends AbstractGCPubSubProcessor {
+public class ConsumeGCPubSub extends AbstractGCPubSubWithProxyProcessor {
 
     private static final List<String> REQUIRED_PERMISSIONS = Collections.singletonList("pubsub.subscriptions.consume");
 
@@ -99,7 +98,7 @@ public class ConsumeGCPubSub extends AbstractGCPubSubProcessor {
     private SubscriberStub subscriber = null;
     private PullRequest pullRequest;
 
-    private AtomicReference<Exception> storedException = new AtomicReference<>();
+    private final AtomicReference<Exception> storedException = new AtomicReference<>();
 
     @OnScheduled
     public void onScheduled(ProcessContext context) {
@@ -191,10 +190,10 @@ public class ConsumeGCPubSub extends AbstractGCPubSubProcessor {
 
     @Override
     public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return Collections.unmodifiableList(Arrays.asList(PROJECT_ID,
-                GCP_CREDENTIALS_PROVIDER_SERVICE,
-                SUBSCRIPTION,
-                BATCH_SIZE));
+        final List<PropertyDescriptor> descriptors = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        descriptors.add(SUBSCRIPTION);
+        descriptors.add(BATCH_SIZE);
+        return Collections.unmodifiableList(descriptors);
     }
 
     @Override
@@ -271,6 +270,7 @@ public class ConsumeGCPubSub extends AbstractGCPubSubProcessor {
     private SubscriberStub getSubscriber(final ProcessContext context) throws IOException {
         final SubscriberStubSettings subscriberStubSettings = SubscriberStubSettings.newBuilder()
                 .setCredentialsProvider(FixedCredentialsProvider.create(getGoogleCredentials(context)))
+                .setTransportChannelProvider(getTransportChannelProvider(context))
                 .build();
 
         return GrpcSubscriberStub.create(subscriberStubSettings);
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/PublishGCPubSub.java b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/PublishGCPubSub.java
index c0646a7028..86545ab5fd 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/PublishGCPubSub.java
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/src/main/java/org/apache/nifi/processors/gcp/pubsub/PublishGCPubSub.java
@@ -86,7 +86,7 @@ import static org.apache.nifi.processors.gcp.pubsub.PubSubAttributes.TOPIC_NAME_
 })
 @SystemResourceConsideration(resource = SystemResource.MEMORY, description = "The entirety of the FlowFile's content "
         + "will be read into memory to be sent as a PubSub message.")
-public class PublishGCPubSub extends AbstractGCPubSubProcessor{
+public class PublishGCPubSub extends AbstractGCPubSubWithProxyProcessor {
     private static final List<String> REQUIRED_PERMISSIONS = Collections.singletonList("pubsub.topics.publish");
 
     public static final PropertyDescriptor TOPIC_NAME = new PropertyDescriptor.Builder()
@@ -104,14 +104,14 @@ public class PublishGCPubSub extends AbstractGCPubSubProcessor{
             .build();
 
     private Publisher publisher = null;
-    private AtomicReference<Exception> storedException = new AtomicReference<>();
+    private final AtomicReference<Exception> storedException = new AtomicReference<>();
 
     @Override
     public List<PropertyDescriptor> getSupportedPropertyDescriptors() {
-        return Collections.unmodifiableList(Arrays.asList(PROJECT_ID,
-                GCP_CREDENTIALS_PROVIDER_SERVICE,
-                TOPIC_NAME,
-                BATCH_SIZE));
+        final List<PropertyDescriptor> descriptors = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        descriptors.add(TOPIC_NAME);
+        descriptors.add(BATCH_SIZE);
+        return Collections.unmodifiableList(descriptors);
     }
 
     @Override
@@ -168,6 +168,7 @@ public class PublishGCPubSub extends AbstractGCPubSubProcessor{
             try {
                 final PublisherStubSettings publisherStubSettings = PublisherStubSettings.newBuilder()
                         .setCredentialsProvider(FixedCredentialsProvider.create(getGoogleCredentials(context)))
+                        .setTransportChannelProvider(getTransportChannelProvider(context))
                         .build();
 
                 final GrpcPublisherStub publisherStub = GrpcPublisherStub.create(publisherStubSettings);
@@ -253,7 +254,7 @@ public class PublishGCPubSub extends AbstractGCPubSubProcessor{
                                         "so routing to retry", new Object[]{topicName, e.getLocalizedMessage()}, e);
                         session.transfer(flowFile, REL_RETRY);
                     } else {
-                        getLogger().error("Failed to publish the message to Google Cloud PubSub topic '{}' due to {}", new Object[]{topicName, e});
+                        getLogger().error("Failed to publish the message to Google Cloud PubSub topic '{}'", topicName, e);
                         session.transfer(flowFile, REL_FAILURE);
                     }
                     context.yield();
@@ -313,6 +314,7 @@ public class PublishGCPubSub extends AbstractGCPubSubProcessor{
 
         return Publisher.newBuilder(getTopicName(context))
                 .setCredentialsProvider(FixedCredentialsProvider.create(getGoogleCredentials(context)))
+                .setChannelProvider(getTransportChannelProvider(context))
                 .setBatchingSettings(BatchingSettings.newBuilder()
                 .setElementCountThreshold(batchSize)
                 .setIsEnabled(true)


[nifi] 01/16: NIFI-9918: 'Parse XML Attributes' property added to XMLReader NIFI-9918: Fixed review findings NIFI-9918: Example added to XMLReader's additionalDetails.html on the new property. Minor documentation fixes.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 45ba416d2a45fea1961be694bb6d00e59624b37b
Author: Peter Gyori <pe...@gmail.com>
AuthorDate: Thu Mar 31 15:16:55 2022 +0200

    NIFI-9918: 'Parse XML Attributes' property added to XMLReader
    NIFI-9918: Fixed review findings
    NIFI-9918: Example added to XMLReader's additionalDetails.html on the new property. Minor documentation fixes.
    
    Signed-off-by: Nathan Gough <th...@gmail.com>
    
    This closes #5964.
---
 .../main/java/org/apache/nifi/xml/XMLReader.java   |  21 +-
 .../java/org/apache/nifi/xml/XMLRecordReader.java  | 120 +++++-----
 .../apache/nifi/xml/inference/XmlRecordSource.java |  20 +-
 .../additionalDetails.html                         | 196 +++++++++++-----
 .../org/apache/nifi/xml/TestInferXmlSchema.java    |  25 ++-
 .../java/org/apache/nifi/xml/TestXMLReader.java    |  21 +-
 .../org/apache/nifi/xml/TestXMLRecordReader.java   | 247 +++++++++++++++------
 7 files changed, 461 insertions(+), 189 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/XMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
index 20ee5195c6..7a5efd49c5 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLReader.java
@@ -29,6 +29,7 @@ import org.apache.nifi.logging.ComponentLog;
 import org.apache.nifi.processor.exception.ProcessException;
 import org.apache.nifi.processor.util.StandardValidators;
 import org.apache.nifi.schema.access.SchemaAccessStrategy;
+import org.apache.nifi.schema.access.SchemaAccessUtils;
 import org.apache.nifi.schema.access.SchemaNotFoundException;
 import org.apache.nifi.schema.inference.SchemaInferenceEngine;
 import org.apache.nifi.schema.inference.RecordSourceFactory;
@@ -106,12 +107,27 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
             .required(false)
             .build();
 
+    public static final PropertyDescriptor PARSE_XML_ATTRIBUTES = new PropertyDescriptor.Builder()
+            .name("parse_xml_attributes")
+            .displayName("Parse XML Attributes")
+            .description("When 'Schema Access Strategy' is 'Infer Schema' and this property is 'true' then XML attributes are parsed and " +
+                    "added to the record as new fields. When the schema is inferred but this property is 'false', " +
+                    "XML attributes and their values are ignored.")
+            .expressionLanguageSupported(ExpressionLanguageScope.NONE)
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(false)
+            .dependsOn(SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, INFER_SCHEMA)
+            .build();
+
+    private volatile boolean parseXmlAttributes;
     private volatile String dateFormat;
     private volatile String timeFormat;
     private volatile String timestampFormat;
 
     @OnEnabled
     public void onEnabled(final ConfigurationContext context) {
+        this.parseXmlAttributes = context.getProperty(PARSE_XML_ATTRIBUTES).asBoolean();
         this.dateFormat = context.getProperty(DateTimeUtils.DATE_FORMAT).getValue();
         this.timeFormat = context.getProperty(DateTimeUtils.TIME_FORMAT).getValue();
         this.timestampFormat = context.getProperty(DateTimeUtils.TIMESTAMP_FORMAT).getValue();
@@ -120,6 +136,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
     @Override
     protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
         final List<PropertyDescriptor> properties = new ArrayList<>(super.getSupportedPropertyDescriptors());
+        properties.add(PARSE_XML_ATTRIBUTES);
         properties.add(SchemaInferenceUtil.SCHEMA_CACHE);
         properties.add(RECORD_FORMAT);
         properties.add(ATTRIBUTE_PREFIX);
@@ -143,7 +160,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
         final RecordSourceFactory<XmlNode> sourceFactory = (variables, contentStream) -> {
             String contentFieldName = trim(context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue());
             contentFieldName = (contentFieldName == null) ? "value" : contentFieldName;
-            return new XmlRecordSource(contentStream, contentFieldName, isMultipleRecords(context, variables));
+            return new XmlRecordSource(contentStream, contentFieldName, isMultipleRecords(context, variables), parseXmlAttributes);
         };
         final Supplier<SchemaInferenceEngine<XmlNode>> schemaInference = () -> new XmlSchemaInference(new TimeValueInference(dateFormat, timeFormat, timestampFormat));
 
@@ -179,7 +196,7 @@ public class XMLReader extends SchemaRegistryService implements RecordReaderFact
         final String contentFieldName = trim(context.getProperty(CONTENT_FIELD_NAME).evaluateAttributeExpressions(variables).getValue());
         final boolean isArray = isMultipleRecords(context, variables);
 
-        return new XMLRecordReader(in, schema, isArray, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger);
+        return new XMLRecordReader(in, schema, isArray, parseXmlAttributes, attributePrefix, contentFieldName, dateFormat, timeFormat, timestampFormat, logger);
     }
 
     private String trim(final String value) {
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/XMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
index 9de1a6253a..9eec76a552 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/XMLRecordReader.java
@@ -57,6 +57,7 @@ public class XMLRecordReader implements RecordReader {
 
     private final ComponentLog logger;
     private final RecordSchema schema;
+    private final boolean parseXmlAttributes;
     private final String attributePrefix;
     private final String contentFieldName;
 
@@ -68,9 +69,11 @@ public class XMLRecordReader implements RecordReader {
     private final Supplier<DateFormat> LAZY_TIME_FORMAT;
     private final Supplier<DateFormat> LAZY_TIMESTAMP_FORMAT;
 
-    public XMLRecordReader(InputStream in, RecordSchema schema, boolean isArray, String attributePrefix, String contentFieldName,
+    public XMLRecordReader(final InputStream in, final RecordSchema schema, final boolean isArray,
+                           final boolean parseXmlAttributes, final String attributePrefix, final String contentFieldName,
                            final String dateFormat, final String timeFormat, final String timestampFormat, final ComponentLog logger) throws MalformedRecordException {
         this.schema = schema;
+        this.parseXmlAttributes = parseXmlAttributes;
         this.attributePrefix = attributePrefix;
         this.contentFieldName = contentFieldName;
         this.logger = logger;
@@ -248,23 +251,10 @@ public class XMLRecordReader implements RecordReader {
     }
 
     private Object parseUnknownField(StartElement startElement, boolean dropUnknown, RecordSchema schema) throws XMLStreamException {
-        // parse attributes
         final Map<String, Object> recordValues = new HashMap<>();
-        final Iterator iterator = startElement.getAttributes();
-        while (iterator.hasNext()) {
-            final Attribute attribute = (Attribute) iterator.next();
-            final String attributeName = attribute.getName().toString();
 
-            if (dropUnknown) {
-                if (schema != null) {
-                    final Optional<RecordField> field = schema.getField(attributeName);
-                    if (field.isPresent()){
-                        recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
-                    }
-                }
-            } else {
-                recordValues.put(attributePrefix == null ? attributeName : attributePrefix + attributeName, attribute.getValue());
-            }
+        if (parseXmlAttributes) {
+            parseAttributesForUnknownField(startElement, schema, dropUnknown, recordValues);
         }
 
         // parse fields
@@ -351,54 +341,33 @@ public class XMLRecordReader implements RecordReader {
         }
     }
 
-    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
-        final Map<String, Object> recordValues = new HashMap<>();
-
-        // parse attributes
+    private void parseAttributesForUnknownField(StartElement startElement, RecordSchema schema, boolean dropUnknown, Map<String, Object> recordValues) {
         final Iterator iterator = startElement.getAttributes();
         while (iterator.hasNext()) {
             final Attribute attribute = (Attribute) iterator.next();
             final String attributeName = attribute.getName().toString();
-
-            final String targetFieldName = attributePrefix == null ? attributeName : attributePrefix + attributeName;
+            final String fieldName = ((attributePrefix == null) ? attributeName : (attributePrefix + attributeName));
 
             if (dropUnknown) {
-                final Optional<RecordField> field = schema.getField(attributeName);
-                if (field.isPresent()){
-
-                    // dropUnknown == true && coerceTypes == true
-                    if (coerceTypes) {
-                        final Object value;
-                        final DataType dataType = field.get().getDataType();
-                        if ((value = parseStringForType(attribute.getValue(), attributeName, dataType)) != null) {
-                            recordValues.put(targetFieldName, value);
-                        }
-
-                    // dropUnknown == true && coerceTypes == false
-                    } else {
-                        recordValues.put(targetFieldName, attribute.getValue());
-                    }
-                }
-            } else {
-
-                // dropUnknown == false && coerceTypes == true
-                if (coerceTypes) {
-                    final Object value;
+                if (schema != null) {
                     final Optional<RecordField> field = schema.getField(attributeName);
                     if (field.isPresent()){
-                        if ((value = parseStringForType(attribute.getValue(), attributeName, field.get().getDataType())) != null) {
-                            recordValues.put(targetFieldName, value);
-                        }
-                    } else {
-                        recordValues.put(targetFieldName, attribute.getValue());
+                        recordValues.put(fieldName, attribute.getValue());
                     }
-
-                    // dropUnknown == false && coerceTypes == false
-                } else {
-                    recordValues.put(targetFieldName, attribute.getValue());
                 }
+            } else {
+                recordValues.put(fieldName, attribute.getValue());
             }
         }
+    }
+
+    private Record parseRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown) throws XMLStreamException, MalformedRecordException {
+        final Map<String, Object> recordValues = new HashMap<>();
+
+        // parse attributes
+        if (parseXmlAttributes) {
+            parseAttributesForRecord(startElement, schema, coerceTypes, dropUnknown, recordValues);
+        }
 
         // parse fields
         StringBuilder content = new StringBuilder();
@@ -506,6 +475,53 @@ public class XMLRecordReader implements RecordReader {
         }
     }
 
+    private void parseAttributesForRecord(StartElement startElement, RecordSchema schema, boolean coerceTypes, boolean dropUnknown, Map<String, Object> recordValues) {
+        final Iterator iterator = startElement.getAttributes();
+        while (iterator.hasNext()) {
+            final Attribute attribute = (Attribute) iterator.next();
+            final String attributeName = attribute.getName().toString();
+
+            final String targetFieldName = attributePrefix == null ? attributeName : attributePrefix + attributeName;
+
+            if (dropUnknown) {
+                final Optional<RecordField> field = schema.getField(attributeName);
+                if (field.isPresent()){
+
+                    // dropUnknown == true && coerceTypes == true
+                    if (coerceTypes) {
+                        final Object value;
+                        final DataType dataType = field.get().getDataType();
+                        if ((value = parseStringForType(attribute.getValue(), attributeName, dataType)) != null) {
+                            recordValues.put(targetFieldName, value);
+                        }
+
+                    // dropUnknown == true && coerceTypes == false
+                    } else {
+                        recordValues.put(targetFieldName, attribute.getValue());
+                    }
+                }
+            } else {
+
+                // dropUnknown == false && coerceTypes == true
+                if (coerceTypes) {
+                    final Object value;
+                    final Optional<RecordField> field = schema.getField(attributeName);
+                    if (field.isPresent()){
+                        if ((value = parseStringForType(attribute.getValue(), attributeName, field.get().getDataType())) != null) {
+                            recordValues.put(targetFieldName, value);
+                        }
+                    } else {
+                        recordValues.put(targetFieldName, attribute.getValue());
+                    }
+
+                    // dropUnknown == false && coerceTypes == false
+                } else {
+                    recordValues.put(targetFieldName, attribute.getValue());
+                }
+            }
+        }
+    }
+
     private void putUnknownTypeInMap(Map<String, Object> values, String fieldName, Object fieldValue) {
         final Object oldValues = values.get(fieldName);
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java
index f77a61422a..8cf8b34899 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/java/org/apache/nifi/xml/inference/XmlRecordSource.java
@@ -39,9 +39,11 @@ public class XmlRecordSource implements RecordSource<XmlNode> {
 
     private final XMLEventReader xmlEventReader;
     private final String contentFieldName;
+    private final boolean parseXmlAttributes;
 
-    public XmlRecordSource(final InputStream in, final String contentFieldName, final boolean ignoreWrapper) throws IOException {
+    public XmlRecordSource(final InputStream in, final String contentFieldName, final boolean ignoreWrapper, final boolean parseXmlAttributes) throws IOException {
         this.contentFieldName = contentFieldName;
+        this.parseXmlAttributes = parseXmlAttributes;
         try {
             final XMLEventReaderProvider provider = new StandardXMLEventReaderProvider();
             xmlEventReader = provider.getEventReader(new StreamSource(in));
@@ -75,11 +77,8 @@ public class XmlRecordSource implements RecordSource<XmlNode> {
         final StringBuilder content = new StringBuilder();
         final Map<String, XmlNode> childNodes = new LinkedHashMap<>();
 
-        final Iterator<?> attributeIterator = startElement.getAttributes();
-        while (attributeIterator.hasNext()) {
-            final Attribute attribute = (Attribute) attributeIterator.next();
-            final String attributeName = attribute.getName().getLocalPart();
-            childNodes.put(attributeName, new XmlTextNode(attributeName, attribute.getValue()));
+        if (parseXmlAttributes) {
+            addXmlAttributesToChildNodes(startElement, childNodes);
         }
 
         while (xmlEventReader.hasNext()) {
@@ -144,4 +143,13 @@ public class XmlRecordSource implements RecordSource<XmlNode> {
 
         return null;
     }
+
+    private void addXmlAttributesToChildNodes(StartElement startElement, Map<String, XmlNode> childNodes) {
+        final Iterator<?> attributeIterator = startElement.getAttributes();
+        while (attributeIterator.hasNext()) {
+            final Attribute attribute = (Attribute) attributeIterator.next();
+            final String attributeName = attribute.getName().getLocalPart();
+            childNodes.put(attributeName, new XmlTextNode(attributeName, attribute.getValue()));
+        }
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
index adcb3cc550..8b4c5b4aaf 100755
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/main/resources/docs/org.apache.nifi.xml.XMLReader/additionalDetails.html
@@ -36,8 +36,7 @@
                 &lt;record&gt;
                   &lt;field1&gt;content&lt;/field1&gt;
                   &lt;field2&gt;content&lt;/field2&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -56,8 +55,7 @@
                     &lt;field1&gt;content&lt;/field1&gt;
                     &lt;field2&gt;content&lt;/field2&gt;
                   &lt;/record&gt;
-                &lt;/root&gt;
-            </pre>
+                &lt;/root&gt;</pre>
     </code>
 
     <h2>Example: Simple Fields</h2>
@@ -73,8 +71,7 @@
                   &lt;record&gt;
                     &lt;simple_field&gt;content&lt;/simple_field&gt;
                   &lt;/record&gt;
-                &lt;/root&gt;
-            </pre>
+                &lt;/root&gt;</pre>
     </code>
 
     <p>
@@ -91,8 +88,7 @@
                   "fields": [
                     { "name": "simple_field", "type": "string" }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
     <h2>Example: Arrays with Simple Fields</h2>
@@ -109,8 +105,7 @@
                   &lt;array_field&gt;content&lt;/array_field&gt;
                   &lt;array_field&gt;content&lt;/array_field&gt;
                   &lt;simple_field&gt;content&lt;/simple_field&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -129,8 +124,7 @@
                     },
                     { "name": "simple_field", "type": "string" }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
     <p>
@@ -151,8 +145,7 @@
             <pre>
                 &lt;record&gt;
                   &lt;field_with_attribute attr="attr_content"&gt;content of field&lt;/field_with_attribute&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -207,8 +200,7 @@
                         ]
                     }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
     <p>
@@ -225,8 +217,7 @@
                         RecordField "prefix_attr" = "attr_content",
                         RecordField "field_name_for_content" = "content of field"
                     )
-                )
-            </pre>
+                )</pre>
     </code>
 
     <p>
@@ -235,6 +226,11 @@
         the property "Field Name for Content" is set.
     </p>
 
+    <p>
+        It is possible that the schema is not provided explicitly, but schema inference is used.
+        For details on XML attributes and schema inference, see "Example: Tags with Attributes and Schema Inference" below.
+    </p>
+
     <h2>Example: Tags within tags</h2>
 
     <p>
@@ -248,8 +244,7 @@
                     &lt;embedded_field&gt;embedded content&lt;/embedded_field&gt;
                     &lt;another_embedded_field&gt;another embedded content&lt;/another_embedded_field&gt;
                   &lt;/field_with_embedded_fields&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -277,8 +272,7 @@
                         ]
                     }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
     <p>
@@ -317,6 +311,79 @@
 
     <h3>XML Attributes and Schema Inference Example 1</h3>
 
+    <p>
+        The simplest case is when XML attributes are ignored completely during schema inference. To achieve this, the "Parse XML Attributes" property in
+        XMLReader is set to "false".
+    </p>
+
+    <p>
+        XMLReader settings:
+    </p>
+
+    <table>
+        <tr>
+            <th>Property Name</th>
+            <th>Property Value</th>
+        </tr>
+        <tr>
+            <td>Schema Access Strategy</td>
+            <td><code>Infer Schema</code></td>
+        </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>false</code></td>
+        </tr>
+        <tr>
+            <td>Expect Records as Array</td>
+            <td><code>false</code></td>
+        </tr>
+        <tr>
+            <td>Field Name for Content</td>
+            <td>not set</td>
+        </tr>
+    </table>
+
+    <p>
+        Input:
+    </p>
+
+    <code>
+            <pre>
+                &lt;record&gt;
+                    &lt;field_with_attribute attr="attr_content"&gt;
+                        content of field
+                    &lt;/field_with_attribute&gt;
+                &lt;/record&gt;</pre>
+    </code>
+
+    <p>
+        Output:
+    </p>
+
+    <code>
+            <pre>
+                &lt;record&gt;
+                    &lt;field_with_attribute&gt;
+                        content of field
+                    &lt;/field_with_attribute&gt;
+                &lt;/record&gt;</pre>
+    </code>
+
+    <p>
+        If "Parse XML Attributes" is "false", the XML attribute is not parsed. Its name does not appear in the inferred schema and its value
+        is ignored. The reader behaves as if the XML attribute was not there.
+    </p>
+
+    <p>
+        Important note: "Field Name for Content" was not set in this example. This could lead to data loss if "field_with_attribute" had child elements,
+        similarly to what is described in "XML Attributes and Schema Inference Example 2" and
+        "XML Attributes and Schema Inference Example 4". To avoid that, "Field Name for Content" needs to be assigned a value that is
+        different from any existing XML tags in the data, like in "XML Attributes and Schema Inference Example 6".
+
+    </p>
+
+    <h3>XML Attributes and Schema Inference Example 2</h3>
+
     <p>
         XMLReader settings:
     </p>
@@ -330,6 +397,10 @@
             <td>Schema Access Strategy</td>
             <td><code>Infer Schema</code></td>
         </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>true</code></td>
+        </tr>
         <tr>
             <td>Expect Records as Array</td>
             <td><code>false</code></td>
@@ -376,10 +447,10 @@
         in the example). So a new field named "value" appears in the schema but no value is assigned to it from the data, thus the field is empty.
         The XML attribute (named "attr") is processed, a field named "attr" is added to the schema and the attribute's value ("attr_content") is assigned to it.
         In a case like this, the parent field's original content is lost and a new field named "value" appears in the schema with no data assigned to it.
-        This is to make sure that no data is overwritten in the record if it already contains a field named "value". More on that case in Example 3 and Example 4.
+        This is to make sure that no data is overwritten in the record if it already contains a field named "value". More on that case in Example 4 and Example 5.
     </p>
 
-    <h3>XML Attributes and Schema Inference Example 2</h3>
+    <h3>XML Attributes and Schema Inference Example 3</h3>
 
     <p>
         In this example, the XMLReader's "Field Name for Content" property is filled with the value "original_content". The input data is the same as
@@ -399,6 +470,10 @@
             <td>Schema Access Strategy</td>
             <td><code>Infer Schema</code></td>
         </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>true</code></td>
+        </tr>
         <tr>
             <td>Expect Records as Array</td>
             <td><code>false</code></td>
@@ -444,7 +519,7 @@
         Since the <code>&lt;field_with_attribute&gt;</code> element had no child-element with the name "original_content", no data is lost.
     </p>
 
-    <h3>XML Attributes and Schema Inference Example 3</h3>
+    <h3>XML Attributes and Schema Inference Example 4</h3>
 
     <p>
         In this example, XMLReader's "Field Name for Content" property is left empty. In the input data, the <code>&lt;field_with_attribute&gt;</code> element
@@ -464,6 +539,10 @@
             <td>Schema Access Strategy</td>
             <td><code>Infer Schema</code></td>
         </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>true</code></td>
+        </tr>
         <tr>
             <td>Expect Records as Array</td>
             <td><code>false</code></td>
@@ -503,16 +582,16 @@
 
     <p>
         The "Field Name for Content" property is not set, and the XML element has a sub-element named "value". The name of the sub-element clashes with the
-        default field name added to the schema by the Schema Inference logic (see Example 1). As seen in the output data, the input XML attribute's value
+        default field name added to the schema by the Schema Inference logic (see Example 2). As seen in the output data, the input XML attribute's value
         is added to the record just like in the previous examples. The value of the <code>&lt;value&gt;</code> element is retained, but the content of the
         <code>&lt;field_with_attribute&gt;</code> that was outside of the sub-element, is lost.
     </p>
 
-    <h3>XML Attributes and Schema Inference Example 4</h3>
+    <h3>XML Attributes and Schema Inference Example 5</h3>
 
     <p>
         In this example, XMLReader's "Field Name for Content" property is given the value "value". In the input data, the <code>&lt;field_with_attribute&gt;</code> element
-       has some content and a sub-element named <code>&lt;value&gt;</code>. The name of the sub-element clashes with the value of the "Field Name for Content" property.
+        has some content and a sub-element named <code>&lt;value&gt;</code>. The name of the sub-element clashes with the value of the "Field Name for Content" property.
     </p>
 
     <p>
@@ -528,6 +607,10 @@
             <td>Schema Access Strategy</td>
             <td><code>Infer Schema</code></td>
         </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>true</code></td>
+        </tr>
         <tr>
             <td>Expect Records as Array</td>
             <td><code>false</code></td>
@@ -571,7 +654,7 @@
         <code>&lt;field_with_attribute&gt;</code> element, and the original content of the <code>&lt;value&gt;</code> element is lost.
     </p>
 
-    <h3>XML Attributes and Schema Inference Example 5</h3>
+    <h3>XML Attributes and Schema Inference Example 6</h3>
 
     <p>
         To avoid losing any data, the XMLReader's "Field Name for Content" property needs to be given a value that does not clash with any sub-element's name
@@ -592,6 +675,10 @@
             <td>Schema Access Strategy</td>
             <td><code>Infer Schema</code></td>
         </tr>
+        <tr>
+            <td>Parse XML Attributes</td>
+            <td><code>true</code></td>
+        </tr>
         <tr>
             <td>Expect Records as Array</td>
             <td><code>false</code></td>
@@ -656,8 +743,7 @@
                     &lt;embedded_field&gt;embedded content 2&lt;/embedded_field&gt;
                     &lt;another_embedded_field&gt;another embedded content 2&lt;/another_embedded_field&gt;
                   &lt;/array_field&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -686,8 +772,7 @@
                       }
                     }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
     <h2>Example: Array in record</h2>
@@ -704,8 +789,7 @@
                     &lt;element&gt;content 2&lt;/element&gt;
                   &lt;/field_enclosing_array&gt;
                   &lt;field_without_array&gt; content 3&lt;/field_without_array&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -737,8 +821,7 @@
                     },
                     { "name": "field_without_array", "type": "string" }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
 
@@ -757,8 +840,7 @@
                     ...
                   &lt;/map_field&gt;
                   &lt;simple_field&gt;content&lt;/simple_field&gt;
-                &lt;/record&gt;
-            </pre>
+                &lt;/record&gt;</pre>
     </code>
 
     <p>
@@ -777,8 +859,7 @@
                     },
                     { "name": "simple_field", "type": "string" }
                   ]
-                }
-            </pre>
+                }</pre>
     </code>
 
 
@@ -794,21 +875,22 @@
     <p>
         A common concern when inferring schemas is how to handle the condition of two values that have different types. For example, consider a FlowFile with the following two records:
     </p>
-    <code><pre>
-<root>
-    <record>
-        <name>John</name>
-        <age>8</age>
-        <values>N/A</values>
-    </record>
-    <record>
-        <name>Jane</name>
-        <age>Ten</age>
-        <values>8</values>
-        <values>Ten</values>
-    </record>
-</root>
-</pre></code>
+    <code>
+        <pre>
+            &lt;root&gt;
+                &lt;record&gt;
+                    &lt;name&gt;John&lt;/name&gt;
+                    &lt;age&gt;8&lt;/age&gt;
+                    &lt;values&gt;N/A&lt;/values&gt;
+                &lt;/record&gt;
+                &lt;record&gt;
+                    &lt;name&gt;Jane&lt;/name&gt;
+                    &lt;age&gt;Ten&lt;/age&gt;
+                    &lt;values&gt;8&lt;/values&gt;
+                    &lt;values&gt;Ten&lt;/values&gt;
+                &lt;/record&gt;
+            &lt;/root&gt;</pre>
+    </code>
 
     <p>
         It is clear that the "name" field will be inferred as a STRING type. However, how should we handle the "age" field? Should the field be an CHOICE between INT and STRING? Should we
@@ -876,7 +958,5 @@
         will typically only be inferred once, regardless of how many Processors handle the data.
     </p>
 
-
-
     </body>
 </html>
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java
index 2a4cd14f10..fa99b447b5 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestInferXmlSchema.java
@@ -48,7 +48,7 @@ public class TestInferXmlSchema {
 
     @Test
     public void testFlatXml() throws IOException {
-        final RecordSchema schema = inferSchema("src/test/resources/xml/person.xml", false);
+        final RecordSchema schema = inferSchema("src/test/resources/xml/person.xml", false, true);
 
         assertEquals(7, schema.getFieldCount());
 
@@ -64,7 +64,7 @@ public class TestInferXmlSchema {
 
     @Test
     public void testFieldsFromAllRecordsIncluded() throws IOException {
-        final RecordSchema schema = inferSchema("src/test/resources/xml/people_nested.xml", true);
+        final RecordSchema schema = inferSchema("src/test/resources/xml/people_nested.xml", true, true);
 
         assertEquals(8, schema.getFieldCount());
 
@@ -94,7 +94,7 @@ public class TestInferXmlSchema {
     @Test
     public void testStringFieldWithAttributes() throws IOException {
         final String contentFieldName = "contentfield";
-        final RecordSchema schema = inferSchema("src/test/resources/xml/TextNodeWithAttribute.xml", contentFieldName, true);
+        final RecordSchema schema = inferSchema("src/test/resources/xml/TextNodeWithAttribute.xml", contentFieldName, true, true);
 
         assertEquals(3, schema.getFieldCount());
 
@@ -110,13 +110,24 @@ public class TestInferXmlSchema {
         assertSame(RecordFieldType.STRING, childSchema.getDataType(contentFieldName).get().getFieldType());
     }
 
-    private RecordSchema inferSchema(final String filename, final boolean ignoreWrapper) throws IOException {
-        return inferSchema(filename, "contentfield", ignoreWrapper);
+    @Test
+    public void testStringFieldWithAttributesIgnored() throws IOException {
+        final RecordSchema schema = inferSchema("src/test/resources/xml/TextNodeWithAttribute.xml", true, false);
+
+        assertEquals(3, schema.getFieldCount());
+
+        assertSame(RecordFieldType.INT, schema.getDataType("num").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("name").get().getFieldType());
+        assertSame(RecordFieldType.STRING, schema.getDataType("software").get().getFieldType());
+    }
+
+    private RecordSchema inferSchema(final String filename, final boolean ignoreWrapper, final boolean parseXMLAttributes) throws IOException {
+        return inferSchema(filename, "contentfield", ignoreWrapper, parseXMLAttributes);
     }
 
-    private RecordSchema inferSchema(final String filename, final String contentFieldName, final boolean ignoreWrapper) throws IOException {
+    private RecordSchema inferSchema(final String filename, final String contentFieldName, final boolean ignoreWrapper, final boolean parseXMLAttributes) throws IOException {
         final File file = new File(filename);
-        final RecordSourceFactory<XmlNode> xmlSourceFactory = (var, in) ->  new XmlRecordSource(in, contentFieldName, ignoreWrapper);
+        final RecordSourceFactory<XmlNode> xmlSourceFactory = (var, in) ->  new XmlRecordSource(in, contentFieldName, ignoreWrapper, parseXMLAttributes);
         final SchemaInferenceEngine<XmlNode> schemaInference = new XmlSchemaInference(timeValueInference);
         final InferSchemaAccessStrategy<XmlNode> inferStrategy = new InferSchemaAccessStrategy<>(xmlSourceFactory, schemaInference, Mockito.mock(ComponentLog.class));
 
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java
index 6904a6f599..d7a3d14316 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLReader.java
@@ -37,7 +37,7 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
-import static junit.framework.TestCase.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
 
 public class TestXMLReader {
 
@@ -279,4 +279,23 @@ public class TestXMLReader {
         String actualContent = out.getContent();
         assertEquals(expectedContent, actualContent);
     }
+
+    @Test
+    public void testInferSchemaIgnoreAttributes() throws InitializationException, IOException {
+        String expectedContent = "MapRecord[{software=Apache NiFi, num=123, name=John Doe}]";
+
+        Map<PropertyDescriptor, String> xmlReaderProperties = new HashMap<>();
+        xmlReaderProperties.put(SchemaAccessUtils.SCHEMA_ACCESS_STRATEGY, SchemaInferenceUtil.INFER_SCHEMA.getValue());
+        xmlReaderProperties.put(XMLReader.RECORD_FORMAT, XMLReader.RECORD_SINGLE.getValue());
+        xmlReaderProperties.put(XMLReader.PARSE_XML_ATTRIBUTES, "false");
+        TestRunner runner = setup(xmlReaderProperties);
+
+        InputStream is = new FileInputStream("src/test/resources/xml/person_record.xml");
+        runner.enqueue(is);
+        runner.run();
+
+        MockFlowFile out = runner.getFlowFilesForRelationship(TestXMLReaderProcessor.SUCCESS).get(0);
+        String actualContent = out.getContent();
+        assertEquals(expectedContent, actualContent);
+    }
 }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java
index 36d7160547..bdf530236a 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-record-serialization-services-bundle/nifi-record-serialization-services/src/test/java/org/apache/nifi/xml/TestXMLRecordReader.java
@@ -55,7 +55,7 @@ public class TestXMLRecordReader {
     @Test
     public void testSingleRecord() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/person.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), false, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
         assertNull(reader.nextRecord());
@@ -64,7 +64,7 @@ public class TestXMLRecordReader {
     @Test
     public void testMap() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_map.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForMap(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForMap(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord();
@@ -85,7 +85,7 @@ public class TestXMLRecordReader {
     @Test
     public void testMapWithRecords() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_map2.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForRecordMap(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForRecordMap(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord();
@@ -104,7 +104,7 @@ public class TestXMLRecordReader {
     @Test
     public void testTagInCharactersSimpleField() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, null}, reader.nextRecord().getValues());
@@ -116,7 +116,7 @@ public class TestXMLRecordReader {
     @Test
     public void testTagInCharactersRecord() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(true, true);
@@ -158,7 +158,7 @@ public class TestXMLRecordReader {
     @Test
     public void testTagInCharactersCoerceTrueDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaWithNestedRecord3(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(true, false);
@@ -205,7 +205,7 @@ public class TestXMLRecordReader {
     @Test
     public void testTagInCharactersCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_tag_in_characters.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
@@ -252,7 +252,7 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecord() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
@@ -264,8 +264,8 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecord2() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema2(), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertNull(reader.nextRecord(true, true).getValue("AGE"));
         assertNull(reader.nextRecord(false, true).getValue("AGE"));
@@ -276,8 +276,8 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecord3() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertEquals(Integer.class, reader.nextRecord(true, true).getValue("AGE").getClass());
         assertEquals(String.class, reader.nextRecord(false, true).getValue("AGE").getClass());
@@ -288,7 +288,7 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.remove(2);
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertEquals(Integer.class, reader.nextRecord(true, false).getValue("AGE").getClass());
@@ -298,8 +298,8 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_no_attributes.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,  true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", "42", "USA"}, reader.nextRecord(false, false).getValues());
         assertArrayEquals(new Object[] {"Ainslie Fletcher", "33", "UK"}, reader.nextRecord(false, false).getValues());
@@ -312,7 +312,7 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord();
@@ -337,7 +337,7 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 "ATTR_", "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord();
@@ -360,8 +360,8 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecordWithAttribute3() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()),
-                true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), true,  true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(true, true);
         assertEquals(null, first.getAsString("ID"));
@@ -382,7 +382,7 @@ public class TestXMLRecordReader {
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.INT.getDataType()));
 
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertEquals(Integer.class, reader.nextRecord(true, true).getValue("ID").getClass());
@@ -395,7 +395,7 @@ public class TestXMLRecordReader {
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.INT.getDataType()));
 
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertEquals(Integer.class, reader.nextRecord(true, false).getValue("ID").getClass());
@@ -404,17 +404,14 @@ public class TestXMLRecordReader {
 
     @Test
     public void testSimpleRecordWithAttribute6() throws IOException, MalformedRecordException {
-        // given
         final InputStream is = new FileInputStream("src/test/resources/xml/people2.xml");
         final List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.DECIMAL.getDecimalDataType(38, 10)));
-        final XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        final XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
-        // when
         final Record record = reader.nextRecord(true, false);
 
-        // then
         assertEquals(BigDecimal.class, record.getValue("ID").getClass());
     }
 
@@ -423,7 +420,7 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ID", RecordFieldType.STRING.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
@@ -462,8 +459,8 @@ public class TestXMLRecordReader {
         final DataType recordType2 = RecordFieldType.RECORD.getRecordDataType(new SimpleRecordSchema(nestedFields2));
         fields.add(new RecordField("AGE", recordType2));
 
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,  true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(true, true);
         assertTrue(first.getValue("NAME") instanceof Record);
@@ -489,7 +486,7 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleTypeWithAttributeAsRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people3.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
@@ -518,7 +515,7 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecordWithHeader() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
@@ -530,7 +527,8 @@ public class TestXMLRecordReader {
     @Test
     public void testSimpleRecordWithHeaderNoValidation() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_with_header_and_comments.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,  true,
+                null, null, dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
         assertArrayEquals(new Object[] {"Ainslie Fletcher", 33, "UK"}, reader.nextRecord().getValues());
@@ -541,7 +539,8 @@ public class TestXMLRecordReader {
     @Test
     public void testInvalidXml() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_invalid.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
         int count = 0;
 
         /*
@@ -566,8 +565,8 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people.xml");
         List<RecordField> fields = getSimpleRecordFields2();
         fields.add(new RecordField("AGE", RecordFieldType.CHOICE.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,  true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record record = reader.nextRecord();
         assertTrue(record.getValue("AGE") instanceof String);
@@ -579,8 +578,8 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ADDRESS", RecordFieldType.CHOICE.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record record = reader.nextRecord();
         assertTrue(record.getValue("ADDRESS") instanceof Record);
@@ -593,7 +592,7 @@ public class TestXMLRecordReader {
     @Test
     public void testNameSpaces() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_namespace.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
@@ -605,7 +604,7 @@ public class TestXMLRecordReader {
     @Test
     public void testCData() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_cdata.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", 42, "USA"}, reader.nextRecord().getValues());
@@ -620,8 +619,8 @@ public class TestXMLRecordReader {
         List<RecordField> fields = getSimpleRecordFields2();
         final DataType recordType = RecordFieldType.RECORD.getRecordDataType(getNestedSchema());
         fields.add(new RecordField("AGE", recordType));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {"Cleve Butler", "USA", null}, reader.nextRecord().getValues());
         assertArrayEquals(new Object[] {"Ainslie Fletcher", "UK", null}, reader.nextRecord().getValues());
@@ -634,8 +633,8 @@ public class TestXMLRecordReader {
         InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml");
         List<RecordField> fields = getSimpleRecordFields();
         fields.add(new RecordField("ADDRESS", RecordFieldType.STRING.getDataType()));
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertNull(reader.nextRecord().getValue("ADDRESS"));
         assertNull(reader.nextRecord().getValue("ADDRESS"));
@@ -646,7 +645,7 @@ public class TestXMLRecordReader {
     @Test
     public void testParseEmptyFields() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord().getValues());
@@ -656,7 +655,7 @@ public class TestXMLRecordReader {
     @Test
     public void testParseEmptyFieldsCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_empty.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertArrayEquals(new Object[] {null, null, null}, reader.nextRecord(false, false).getValues());
@@ -667,27 +666,29 @@ public class TestXMLRecordReader {
     public void testEmptyStreamAsSingleRecord() {
         InputStream is = new ByteArrayInputStream(new byte[0]);
         assertThrows(MalformedRecordException.class,
-                () -> new XMLRecordReader(is, getSimpleSchema(), false, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)));
+                () -> new XMLRecordReader(is, getSimpleSchema(), false,  true,
+                        null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)));
     }
 
     @Test
     public void testEmptyStreamAsArray() {
         InputStream is = new ByteArrayInputStream(new byte[0]);
         assertThrows(MalformedRecordException.class,
-                () -> new XMLRecordReader(is, getSimpleSchema(), true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)));
+                () -> new XMLRecordReader(is, getSimpleSchema(), true, true,
+                        null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)));
     }
 
     @Test
     public void testEmptyStreamWIthXmlHeader() {
         InputStream is = new ByteArrayInputStream(("<?xml version=\"1.0\" encoding=\"utf-8\"?>").getBytes());
-        assertThrows(MalformedRecordException.class, () -> new XMLRecordReader(is, getSimpleSchema(), true,
+        assertThrows(MalformedRecordException.class, () -> new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class)));
     }
 
     @Test
     public void testParseEmptyArray() throws IOException, MalformedRecordException {
         InputStream is = new ByteArrayInputStream("<root></root>".getBytes());
-        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSimpleSchema(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         assertNull(reader.nextRecord());
@@ -697,7 +698,7 @@ public class TestXMLRecordReader {
     public void testNestedRecord() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml");
         RecordSchema schema = getSchemaWithNestedRecord();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Object[] valuesFirstRecord = reader.nextRecord().getValues();
@@ -721,7 +722,7 @@ public class TestXMLRecordReader {
     public void testNestedRecordCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_nested.xml");
         RecordSchema schema = getSchemaWithNestedRecord();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
@@ -767,7 +768,7 @@ public class TestXMLRecordReader {
 
         // Fields "AGE" and "ADDRESS/CITY" are not defined here
         RecordSchema schema = getSchemaWithNestedRecord2();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord(true, true);
@@ -809,7 +810,7 @@ public class TestXMLRecordReader {
 
         // Fields "AGE" and "ADDRESS/CITY" are not defined here
         RecordSchema schema = getSchemaWithNestedRecord2();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord(false, true);
@@ -851,7 +852,7 @@ public class TestXMLRecordReader {
 
         // Fields "AGE" and "ADDRESS/CITY" are not defined here
         RecordSchema schema = getSchemaWithNestedRecord2();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord(true, false);
@@ -901,7 +902,7 @@ public class TestXMLRecordReader {
 
         // Fields "AGE" and "ADDRESS/CITY" are not defined here
         RecordSchema schema = getSchemaWithNestedRecord2();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord(false, false);
@@ -950,7 +951,7 @@ public class TestXMLRecordReader {
     public void testSimpleArray() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml");
         RecordSchema schema = getSchemaWithSimpleArray();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord();
@@ -986,7 +987,7 @@ public class TestXMLRecordReader {
     public void testSimpleArrayCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_array_simple.xml");
         RecordSchema schema = getSchemaWithSimpleArray();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true,
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
@@ -1021,8 +1022,8 @@ public class TestXMLRecordReader {
     public void testNestedArrayInNestedRecord() throws IOException, MalformedRecordException {
         InputStream is = new FileInputStream("src/test/resources/xml/people_array.xml");
         RecordSchema schema = getSchemaWithNestedArray();
-        XMLRecordReader reader = new XMLRecordReader(is, schema, true, null,
-                "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, schema, true, true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record firstRecord = reader.nextRecord();
         Object[] valuesFirstRecord = firstRecord.getValues();
@@ -1059,7 +1060,7 @@ public class TestXMLRecordReader {
     public void testDeeplyNestedArraysAndRecords() throws IOException, MalformedRecordException {
         // test records in nested arrays
         InputStream is = new FileInputStream("src/test/resources/xml/people_complex1.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(true, true);
@@ -1098,7 +1099,7 @@ public class TestXMLRecordReader {
     public void testDeeplyNestedArraysAndRecords2() throws IOException, MalformedRecordException {
         // test multiply nested arrays and records (recursion)
         InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord();
@@ -1154,7 +1155,7 @@ public class TestXMLRecordReader {
     public void testDeeplyNestedArraysAndRecordsCoerceFalseDropTrue() throws IOException, MalformedRecordException {
         // test multiply nested arrays and records (recursion)
         InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true,
+        XMLRecordReader reader = new XMLRecordReader(is, getSchemaForComplexData2(), true, true,
                 null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, true);
@@ -1210,8 +1211,8 @@ public class TestXMLRecordReader {
     public void testDeeplyNestedArraysAndRecordsCoerceFalseDropFalse() throws IOException, MalformedRecordException {
         // test multiply nested arrays and records (recursion)
         InputStream is = new FileInputStream("src/test/resources/xml/people_complex2.xml");
-        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()),
-                true, null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(Collections.emptyList()), true,  true,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
 
         Record first = reader.nextRecord(false, false);
         assertEquals("1", first.getValue("ID"));
@@ -1267,6 +1268,110 @@ public class TestXMLRecordReader {
                 .getValue("ID"));
     }
 
+    @Test
+    public void testSimpleTypeWithAttributesIgnored1() throws IOException, MalformedRecordException {
+        boolean parseXMLAttributes = false;
+        boolean coerceTypes = true;
+        boolean dropUnknownFields = true;
+
+        List<Record> records = simpleTypeWithAttributesIgnored(parseXMLAttributes, coerceTypes, dropUnknownFields);
+
+        Record first = records.get(0);
+        Record second = records.get(1);
+
+        assertTrue(first.getValue("NAME") instanceof String);
+        assertEquals("Cleve Butler", first.getValue("NAME"));
+        assertTrue(first.getValue("AGE") instanceof Integer);
+        assertEquals(42, first.getValue("AGE"));
+        assertEquals(2, first.toMap().size());
+
+        assertTrue(second.getValue("NAME") instanceof String);
+        assertEquals("Ainslie Fletcher", second.getValue("NAME"));
+        assertTrue(second.getValue("AGE") instanceof  Integer);
+        assertEquals(33, second.getValue("AGE"));
+        assertEquals(2, second.toMap().size());
+    }
+
+    @Test
+    public void testSimpleTypeWithAttributesIgnored2() throws IOException, MalformedRecordException {
+        boolean parseXMLAttributes = false;
+        boolean coerceTypes = false;
+        boolean dropUnknownFields = true;
+
+        List<Record> records = simpleTypeWithAttributesIgnored(parseXMLAttributes, coerceTypes, dropUnknownFields);
+
+        Record first = records.get(0);
+        Record second = records.get(1);
+
+        assertTrue(first.getValue("NAME") instanceof String);
+        assertEquals("Cleve Butler", first.getValue("NAME"));
+        assertTrue(first.getValue("AGE") instanceof String);
+        assertEquals("42", first.getValue("AGE"));
+        assertEquals(2, first.toMap().size());
+
+        assertTrue(second.getValue("NAME") instanceof String);
+        assertEquals("Ainslie Fletcher", second.getValue("NAME"));
+        assertTrue(second.getValue("AGE") instanceof  String);
+        assertEquals("33", second.getValue("AGE"));
+        assertEquals(2, second.toMap().size());
+    }
+
+    @Test
+    public void testSimpleTypeWithAttributesIgnored3() throws IOException, MalformedRecordException {
+        boolean parseXMLAttributes = false;
+        boolean coerceTypes = true;
+        boolean dropUnknownFields = false;
+
+        List<Record> records = simpleTypeWithAttributesIgnored(parseXMLAttributes, coerceTypes, dropUnknownFields);
+
+        Record first = records.get(0);
+        Record second = records.get(1);
+
+        assertTrue(first.getValue("NAME") instanceof String);
+        assertEquals("Cleve Butler", first.getValue("NAME"));
+        assertTrue(first.getValue("AGE") instanceof Integer);
+        assertEquals(42, first.getValue("AGE"));
+        assertTrue(first.getValue("COUNTRY") instanceof String);
+        assertEquals("USA", first.getValue("COUNTRY"));
+        assertEquals(3, first.toMap().size());
+
+        assertTrue(second.getValue("NAME") instanceof String);
+        assertEquals("Ainslie Fletcher", second.getValue("NAME"));
+        assertTrue(second.getValue("AGE") instanceof  Integer);
+        assertEquals(33, second.getValue("AGE"));
+        assertTrue(second.getValue("COUNTRY") instanceof  String);
+        assertEquals("UK", second.getValue("COUNTRY"));
+        assertEquals(3, second.toMap().size());
+    }
+
+    @Test
+    public void testSimpleTypeWithAttributesIgnored4() throws IOException, MalformedRecordException {
+        boolean parseXMLAttributes = false;
+        boolean coerceTypes = false;
+        boolean dropUnknownFields = false;
+
+        List<Record> records = simpleTypeWithAttributesIgnored(parseXMLAttributes, coerceTypes, dropUnknownFields);
+
+        Record first = records.get(0);
+        Record second = records.get(1);
+
+        assertTrue(first.getValue("NAME") instanceof String);
+        assertEquals("Cleve Butler", first.getValue("NAME"));
+        assertTrue(first.getValue("AGE") instanceof String);
+        assertEquals("42", first.getValue("AGE"));
+        assertTrue(first.getValue("COUNTRY") instanceof String);
+        assertEquals("USA", first.getValue("COUNTRY"));
+        assertEquals(3, first.toMap().size());
+
+        assertTrue(second.getValue("NAME") instanceof String);
+        assertEquals("Ainslie Fletcher", second.getValue("NAME"));
+        assertTrue(second.getValue("AGE") instanceof  String);
+        assertEquals("33", second.getValue("AGE"));
+        assertTrue(second.getValue("COUNTRY") instanceof  String);
+        assertEquals("UK", second.getValue("COUNTRY"));
+        assertEquals(3, second.toMap().size());
+    }
+
     private List<RecordField> getSimpleRecordFields() {
         final List<RecordField> fields = new ArrayList<>();
         fields.add(new RecordField("NAME", RecordFieldType.STRING.getDataType()));
@@ -1449,4 +1554,20 @@ public class TestXMLRecordReader {
         }};
         return new SimpleRecordSchema(fields);
     }
+
+    private List<Record> simpleTypeWithAttributesIgnored(boolean parseXMLAttributes, boolean coerceTypes, boolean dropunknownFields) throws IOException, MalformedRecordException {
+        InputStream is = new FileInputStream("src/test/resources/xml/people3.xml");
+
+        final List<RecordField> fields = new ArrayList<>();
+        fields.add(new RecordField("NAME", RecordFieldType.STRING.getDataType()));
+        fields.add(new RecordField("AGE", RecordFieldType.INT.getDataType()));
+
+        XMLRecordReader reader = new XMLRecordReader(is, new SimpleRecordSchema(fields), true,  parseXMLAttributes,
+                null, "CONTENT", dateFormat, timeFormat, timestampFormat, Mockito.mock(ComponentLog.class));
+
+        List<Record> records = new ArrayList<>(2);
+        records.add(reader.nextRecord(coerceTypes, dropunknownFields));
+        records.add(reader.nextRecord(coerceTypes, dropunknownFields));
+        return records;
+    }
 }


[nifi] 09/16: NIFI-8533: Replace deprecated jython-shaded with jython-slim

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e3b85d4b145bccf6aa7ea9af4455cdb4adf88826
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Wed Nov 17 15:39:49 2021 -0500

    NIFI-8533: Replace deprecated jython-shaded with jython-slim
    
    This closes #5531
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../nifi-scripting-bundle/nifi-scripting-processors/pom.xml         | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)

diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
index e19c727146..8ca1385a5a 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
@@ -76,9 +76,9 @@
             <version>2.4.0</version>
         </dependency>
         <dependency>
-            <groupId>org.scijava</groupId>
-            <artifactId>jython-shaded</artifactId>
-            <version>2.7.1.1</version>
+            <groupId>org.python</groupId>
+            <artifactId>jython-slim</artifactId>
+            <version>2.7.2</version>
         </dependency>
         <dependency>
             <groupId>org.luaj</groupId>


[nifi] 13/16: NIFI-9984 Allow 200-series responses in OAuth2 Access Token Provider

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 237b0abbf3d3e558cd8a0b889a0b206271258983
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed May 4 11:13:57 2022 -0500

    NIFI-9984 Allow 200-series responses in OAuth2 Access Token Provider
    
    This closes #6016
    
    Signed-off-by: Mike Thomsen <mt...@apache.org>
---
 .../nifi/oauth2/StandardOauth2AccessTokenProvider.java  | 17 ++++++++---------
 .../oauth2/StandardOauth2AccessTokenProviderTest.java   | 15 +++++++++------
 2 files changed, 17 insertions(+), 15 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
index c09ecc9e96..af3a96dd68 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
@@ -315,19 +315,18 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
         this.accessDetails = getAccessDetails(refreshRequest);
     }
 
-    private AccessToken getAccessDetails(Request newRequest) {
+    private AccessToken getAccessDetails(final Request newRequest) {
         try {
-            Response response = httpClient.newCall(newRequest).execute();
-            String responseBody = response.body().string();
-            if (response.code() != 200) {
+            final Response response = httpClient.newCall(newRequest).execute();
+            final String responseBody = response.body().string();
+            if (response.isSuccessful()) {
+                getLogger().debug("OAuth2 Access Token retrieved [HTTP {}]", response.code());
+                return ACCESS_DETAILS_MAPPER.readValue(responseBody, AccessToken.class);
+            } else {
                 getLogger().error(String.format("OAuth2 access token request failed [HTTP %d], response:%n%s", response.code(), responseBody));
                 throw new ProcessException(String.format("OAuth2 access token request failed [HTTP %d]", response.code()));
             }
-
-            AccessToken accessDetails = ACCESS_DETAILS_MAPPER.readValue(responseBody, AccessToken.class);
-
-            return accessDetails;
-        } catch (IOException e) {
+        } catch (final IOException e) {
             throw new UncheckedIOException("OAuth2 access token request failed", e);
         }
     }
diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/test/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProviderTest.java b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/test/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProviderTest.java
index cbc485aaf6..20054bcad3 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/test/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProviderTest.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/test/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProviderTest.java
@@ -64,6 +64,9 @@ public class StandardOauth2AccessTokenProviderTest {
     private static final String CLIENT_SECRET = "clientSecret";
     private static final long FIVE_MINUTES = 300;
 
+    private static final int HTTP_OK = 200;
+    private static final int HTTP_ACCEPTED = 201;
+
     private StandardOauth2AccessTokenProvider testSubject;
 
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
@@ -146,7 +149,7 @@ public class StandardOauth2AccessTokenProviderTest {
 
         // GIVEN
         Response response = buildResponse(
-            200,
+            HTTP_OK,
             "{ \"access_token\":\"" + accessTokenValue + "\" }"
         );
 
@@ -166,12 +169,12 @@ public class StandardOauth2AccessTokenProviderTest {
         String expectedToken = "second_token";
 
         Response response1 = buildResponse(
-            200,
+                HTTP_OK,
             "{ \"access_token\":\"" + firstToken + "\", \"expires_in\":\"0\", \"refresh_token\":\"not_checking_in_this_test\" }"
         );
 
         Response response2 = buildResponse(
-            200,
+                HTTP_OK,
             "{ \"access_token\":\"" + expectedToken + "\" }"
         );
 
@@ -230,7 +233,7 @@ public class StandardOauth2AccessTokenProviderTest {
         String expectedToken = "expected_token";
 
         Response successfulAcquireResponse = buildResponse(
-            200,
+                HTTP_ACCEPTED,
             "{ \"access_token\":\"" + expectedToken + "\", \"expires_in\":\"0\", \"refresh_token\":\"not_checking_in_this_test\" }"
         );
 
@@ -319,7 +322,7 @@ public class StandardOauth2AccessTokenProviderTest {
 
         Response errorRefreshResponse = buildResponse(500, expectedRefreshErrorResponse);
         Response successfulAcquireResponse = buildResponse(
-            200,
+                HTTP_OK,
             "{ \"access_token\":\"" + expectedToken + "\", \"expires_in\":\"0\", \"refresh_token\":\"not_checking_in_this_test\" }"
         );
 
@@ -358,7 +361,7 @@ public class StandardOauth2AccessTokenProviderTest {
 
     private Response buildSuccessfulInitResponse() {
         return buildResponse(
-            200,
+                HTTP_OK,
             "{ \"access_token\":\"exists_but_value_irrelevant\", \"expires_in\":\"0\", \"refresh_token\":\"not_checking_in_this_test\" }"
         );
     }


[nifi] 03/16: NIFI-9944 Configured maven-dependency-plugin for JavaScript test JAR

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 028982ce0b26ebab1eb3be72dd9c977a32c732a4
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Tue Apr 26 20:48:46 2022 -0500

    NIFI-9944 Configured maven-dependency-plugin for JavaScript test JAR
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #5997
---
 .../nifi-scripting-processors/pom.xml              |  26 +++++++++
 .../processors/script/TestInvokeJavascript.java    |  62 ++++++++++-----------
 .../src/test/resources/jar/commons-math3-3.2.jar   | Bin 1692782 -> 0 bytes
 3 files changed, 55 insertions(+), 33 deletions(-)

diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
index 678bd9865c..e19c727146 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/pom.xml
@@ -163,6 +163,32 @@
                     </excludes>
                 </configuration>
             </plugin>
+            <plugin>
+                <groupId>org.apache.maven.plugins</groupId>
+                <artifactId>maven-dependency-plugin</artifactId>
+                <version>3.3.0</version>
+                <executions>
+                    <execution>
+                        <id>copy</id>
+                        <phase>process-test-resources</phase>
+                        <goals>
+                            <goal>copy</goal>
+                        </goals>
+                        <configuration>
+                            <artifactItems>
+                                <artifactItem>
+                                    <groupId>org.apache.commons</groupId>
+                                    <artifactId>commons-math3</artifactId>
+                                    <version>3.6.1</version>
+                                    <type>jar</type>
+                                    <overWrite>true</overWrite>
+                                    <outputDirectory>${project.build.testOutputDirectory}/jar</outputDirectory>
+                                </artifactItem>
+                            </artifactItems>
+                        </configuration>
+                    </execution>
+                </executions>
+            </plugin>
         </plugins>
     </build>
 
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
index c56612bd2a..6955c57349 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
@@ -20,9 +20,6 @@ import org.apache.nifi.components.PropertyDescriptor;
 import org.apache.nifi.processor.Relationship;
 import org.apache.nifi.script.ScriptingComponentUtils;
 import org.apache.nifi.util.MockFlowFile;
-import org.apache.nifi.util.MockProcessContext;
-import org.apache.nifi.util.MockProcessorInitializationContext;
-import org.apache.nifi.util.MockValidationContext;
 import org.apache.nifi.util.TestRunner;
 import org.apache.nifi.util.TestRunners;
 import org.junit.jupiter.api.BeforeEach;
@@ -30,8 +27,13 @@ import org.junit.jupiter.api.Test;
 import org.junit.jupiter.api.condition.DisabledForJreRange;
 import org.junit.jupiter.api.condition.JRE;
 
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.net.URL;
 import java.nio.charset.StandardCharsets;
+import java.nio.file.Paths;
 import java.util.List;
+import java.util.Objects;
 import java.util.Set;
 
 import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -56,9 +58,7 @@ public class TestInvokeJavascript extends BaseScriptTest {
      */
     @Test
     public void testReadFlowFileContentAndStoreInFlowFileAttribute() {
-        runner.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
-        runner.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        runner.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
+        setScriptProperties();
 
         runner.assertValid();
         runner.enqueue("test content".getBytes(StandardCharsets.UTF_8));
@@ -78,20 +78,10 @@ public class TestInvokeJavascript extends BaseScriptTest {
      */
     @Test
     public void testScriptDefinedAttribute() {
-        InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
-        MockProcessContext context = new MockProcessContext(processor);
-        MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
-
-        processor.initialize(initContext);
-
-        context.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
-        context.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
-        // State Manger is unused, and a null reference is specified
-        processor.customValidate(new MockValidationContext(context));
-        processor.setup(context);
+        setScriptProperties();
+        runner.assertValid();
 
-        List<PropertyDescriptor> descriptors = processor.getSupportedPropertyDescriptors();
+        List<PropertyDescriptor> descriptors = runner.getProcessor().getPropertyDescriptors();
         assertNotNull(descriptors);
         assertTrue(descriptors.size() > 0);
         boolean found = false;
@@ -113,21 +103,10 @@ public class TestInvokeJavascript extends BaseScriptTest {
      */
     @Test
     public void testScriptDefinedRelationshipWithExternalJar() {
-        InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
-        MockProcessContext context = new MockProcessContext(processor);
-        MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
-
-        processor.initialize(initContext);
-
-        context.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
-        context.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
-
-        // State Manger is unused, and a null reference is specified
-        processor.customValidate(new MockValidationContext(context));
-        processor.setup(context);
+        setScriptProperties();
+        runner.assertValid();
 
-        Set<Relationship> relationships = processor.getRelationships();
+        Set<Relationship> relationships = runner.getProcessor().getRelationships();
         assertNotNull(relationships);
         assertTrue(relationships.size() > 0);
         boolean found = false;
@@ -188,4 +167,21 @@ public class TestInvokeJavascript extends BaseScriptTest {
         runner.setProperty(ScriptingComponentUtils.SCRIPT_BODY, "");
         runner.assertNotValid();
     }
+
+    private void setScriptProperties() {
+        runner.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
+        runner.setProperty(ScriptingComponentUtils.SCRIPT_FILE, getResource("/javascript/test_reader.js"));
+        runner.setProperty(ScriptingComponentUtils.MODULES, getResource("/jar"));
+    }
+
+    private String getResource(final String resourcePath) {
+        final URL resourceUrl = Objects.requireNonNull(TestInvokeJavascript.class.getResource(resourcePath), resourcePath);
+        final URI resourceUri;
+        try {
+            resourceUri = resourceUrl.toURI();
+        } catch (final URISyntaxException e) {
+            throw new RuntimeException(e);
+        }
+        return Paths.get(resourceUri).toString();
+    }
 }
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar
deleted file mode 100644
index f8b7db295b..0000000000
Binary files a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar and /dev/null differ


[nifi] 12/16: NIFI-9988 Corrected Property Decryption for Authorizers and Providers

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 409566c97d69837c73bd4318a35f21d495e7e629
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed May 4 12:48:34 2022 -0500

    NIFI-9988 Corrected Property Decryption for Authorizers and Providers
    
    - Updated Protection Scheme Resolver to support both Name matching and Path matching
    
    Signed-off-by: Nathan Gough <th...@gmail.com>
    
    This closes #6017.
---
 .../nifi/properties/scheme/StandardProtectionSchemeResolver.java | 4 +++-
 .../properties/scheme/StandardProtectionSchemeResolverTest.java  | 9 +++++++++
 2 files changed, 12 insertions(+), 1 deletion(-)

diff --git a/nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolver.java b/nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolver.java
index 0c797b3b93..44557963e4 100644
--- a/nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolver.java
+++ b/nifi-commons/nifi-property-protection-factory/src/main/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolver.java
@@ -37,7 +37,9 @@ public class StandardProtectionSchemeResolver implements ProtectionSchemeResolve
     public ProtectionScheme getProtectionScheme(final String scheme) {
         Objects.requireNonNull(scheme, "Scheme required");
         return Arrays.stream(PropertyProtectionScheme.values())
-                .filter(propertyProtectionScheme -> propertyProtectionScheme.name().equals(scheme))
+                .filter(propertyProtectionScheme ->
+                        propertyProtectionScheme.name().equals(scheme) || scheme.startsWith(propertyProtectionScheme.getPath())
+                )
                 .findFirst()
                 .orElseThrow(() -> new SensitivePropertyProtectionException(String.format("Protection Scheme [%s] not supported", scheme)));
     }
diff --git a/nifi-commons/nifi-property-protection-factory/src/test/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolverTest.java b/nifi-commons/nifi-property-protection-factory/src/test/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolverTest.java
index 9cfc4994f7..c8893b2231 100644
--- a/nifi-commons/nifi-property-protection-factory/src/test/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolverTest.java
+++ b/nifi-commons/nifi-property-protection-factory/src/test/java/org/apache/nifi/properties/scheme/StandardProtectionSchemeResolverTest.java
@@ -30,6 +30,8 @@ public class StandardProtectionSchemeResolverTest {
 
     private static final String AES_GCM_PATH = "aes/gcm";
 
+    private static final String AES_GCM_256_PATH = "aes/gcm/256";
+
     private static final String UNKNOWN = "UNKNOWN";
 
     private StandardProtectionSchemeResolver resolver;
@@ -46,6 +48,13 @@ public class StandardProtectionSchemeResolverTest {
         assertEquals(AES_GCM_PATH, protectionScheme.getPath());
     }
 
+    @Test
+    public void getProtectionSchemeAesGcm256Found() {
+        final ProtectionScheme protectionScheme = resolver.getProtectionScheme(AES_GCM_256_PATH);
+        assertNotNull(protectionScheme);
+        assertEquals(AES_GCM_PATH, protectionScheme.getPath());
+    }
+
     @Test
     public void getProtectionSchemeUnknownNotFound() {
         final SensitivePropertyProtectionException exception = assertThrows(SensitivePropertyProtectionException.class, () -> resolver.getProtectionScheme(UNKNOWN));


[nifi] 05/16: NIFI-9976 Upgraded json-smart to 2.4.8

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit e80e099265dd54f4e32961603d6af6b12000bcda
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Thu Apr 28 21:35:15 2022 -0500

    NIFI-9976 Upgraded json-smart to 2.4.8
    
    - Replaced nifi-framework-bundle managed dependency to root managed dependency
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #6005.
---
 nifi-nar-bundles/nifi-framework-bundle/pom.xml | 5 -----
 pom.xml                                        | 6 ++++++
 2 files changed, 6 insertions(+), 5 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/pom.xml b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
index 16c176e510..1ae7899af8 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-framework-bundle/pom.xml
@@ -315,11 +315,6 @@
                 <artifactId>oauth2-oidc-sdk</artifactId>
                 <version>9.10.2</version>
             </dependency>
-            <dependency>
-                <groupId>net.minidev</groupId>
-                <artifactId>json-smart</artifactId>
-                <version>2.4.7</version>
-            </dependency>
             <dependency>
                 <groupId>com.nimbusds</groupId>
                 <artifactId>lang-tag</artifactId>
diff --git a/pom.xml b/pom.xml
index 552de348f3..ea4278658c 100644
--- a/pom.xml
+++ b/pom.xml
@@ -115,6 +115,7 @@
         <jackson.bom.version>2.13.2.20220328</jackson.bom.version>
         <jaxb.runtime.version>2.3.5</jaxb.runtime.version>
         <jakarta.xml.bind-api.version>2.3.3</jakarta.xml.bind-api.version>
+        <json.smart.version>2.4.8</json.smart.version>
         <nifi.groovy.version>3.0.8</nifi.groovy.version>
         <surefire.version>3.0.0-M5</surefire.version>
         <!-- The Hadoop version used by nifi-hadoop-libraries-nar and any NARs that depend on it, other NARs that need
@@ -514,6 +515,11 @@
                 <artifactId>aspectjweaver</artifactId>
                 <version>${aspectj.version}</version>
             </dependency>
+            <dependency>
+                <groupId>net.minidev</groupId>
+                <artifactId>json-smart</artifactId>
+                <version>${json.smart.version}</version>
+            </dependency>
             <dependency>
                 <groupId>com.fasterxml.jackson</groupId>
                 <artifactId>jackson-bom</artifactId>


[nifi] 15/16: NIFI-9998: This closes #6023. Upgrade Hive3 to 3.1.3

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 51610078b2ff1ae0e1ca8565da03aaf3084ea688
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Fri May 6 14:50:40 2022 -0400

    NIFI-9998: This closes #6023. Upgrade Hive3 to 3.1.3
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 .../nifi-hive-bundle/nifi-hive3-processors/pom.xml | 77 ++--------------------
 nifi-nar-bundles/nifi-hive-bundle/pom.xml          |  2 +-
 2 files changed, 8 insertions(+), 71 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 a6faff95ca..a26514e4e9 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
@@ -109,6 +109,10 @@
                     <groupId>org.apache.logging.log4j</groupId>
                     <artifactId>log4j-1.2-api</artifactId>
                 </exclusion>
+                <exclusion>
+                    <groupId>org.apache.logging.log4j</groupId>
+                    <artifactId>log4j-core</artifactId>
+                </exclusion>
                 <exclusion>
                     <groupId>org.apache.logging.log4j</groupId>
                     <artifactId>log4j-web</artifactId>
@@ -123,7 +127,6 @@
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-exec</artifactId>
             <version>${hive.version}</version>
-            <classifier>core</classifier>
             <exclusions>
                 <exclusion>
                     <groupId>log4j</groupId>
@@ -144,80 +147,14 @@
                 </exclusion>
                 <exclusion>
                     <groupId>org.apache.logging.log4j</groupId>
-                    <artifactId>log4j-slf4j-impl</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-
-        <!-- hive-exec:core doesn't contain these dependencies (as opposed to regular hive-exec) so these need to be pulled in -->
-        <dependency>
-            <groupId>com.esotericsoftware</groupId>
-            <artifactId>kryo-shaded</artifactId>
-            <version>3.0.3</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.commons</groupId>
-            <artifactId>commons-lang3</artifactId>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.avro</groupId>
-            <artifactId>avro-mapred</artifactId>
-            <classifier>hadoop2</classifier>
-            <version>1.8.1</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>org.mortbay.jetty</groupId>
-                    <artifactId>servlet-api</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>com.googlecode.javaewah</groupId>
-            <artifactId>JavaEWAH</artifactId>
-            <version>0.3.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-spark-client</artifactId>
-            <version>${hive.version}</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
-                </exclusion>
-            </exclusions>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.orc</groupId>
-            <artifactId>orc-tools</artifactId>
-            <version>1.5.6</version>
-            <exclusions>
-                <exclusion>
-                    <groupId>log4j</groupId>
-                    <artifactId>log4j</artifactId>
+                    <artifactId>log4j-core</artifactId>
                 </exclusion>
                 <exclusion>
-                    <groupId>commons-logging</groupId>
-                    <artifactId>commons-logging</artifactId>
-                </exclusion>
-                <exclusion>
-                    <artifactId>hadoop-common</artifactId>
-                    <groupId>org.apache.hadoop</groupId>
+                    <groupId>org.apache.logging.log4j</groupId>
+                    <artifactId>log4j-slf4j-impl</artifactId>
                 </exclusion>
             </exclusions>
         </dependency>
-        <dependency>
-            <groupId>org.jodd</groupId>
-            <artifactId>jodd-core</artifactId>
-            <version>3.5.2</version>
-        </dependency>
-        <dependency>
-            <groupId>org.apache.hive</groupId>
-            <artifactId>hive-storage-api</artifactId>
-            <version>2.7.0</version>
-        </dependency>
-        <!-- /hive-exec:core doesn't contain these dependencies (as opposed to regular hive-exec) so these need to be pulled in -->
-
         <dependency>
             <groupId>org.apache.hive</groupId>
             <artifactId>hive-streaming</artifactId>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
index 896bc9e40c..eeca6135c7 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/pom.xml
@@ -105,7 +105,7 @@
         <hive11.hadoop.version>2.6.2</hive11.hadoop.version>
         <hive12.version>1.2.1</hive12.version>
         <hive12.hadoop.version>2.6.2</hive12.hadoop.version>
-        <hive3.version>3.1.2</hive3.version>
+        <hive3.version>3.1.3</hive3.version>
         <hive.version>${hive3.version}</hive.version>
         <calcite.version>1.27.0</calcite.version>
         <calcite.avatica.version>1.6.0</calcite.avatica.version>


[nifi] 08/16: NIFI-9978: - Updating the condition under which we show the Download Flow menu items.

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit b114ee0f46978f14fe910930d0db18ea1bfefd55
Author: Matt Gilman <ma...@gmail.com>
AuthorDate: Fri Apr 29 16:19:07 2022 -0400

    NIFI-9978:
    - Updating the condition under which we show the Download Flow menu items.
    
    This closes #6008.
    
    Signed-off-by: Peter Turcsanyi <tu...@apache.org>
---
 .../nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
index bd0460ecb0..6673fbc062 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-web/nifi-web-ui/src/main/webapp/js/nf/canvas/nf-context-menu.js
@@ -862,8 +862,8 @@
         {id: 'group-menu-item', condition: canGroup, menuItem: {clazz: 'icon icon-group', text: 'Group', action: 'group'}},
         {separator: true},
         {id: 'download-menu-item', groupMenuItem: {clazz: 'fa', text: 'Download flow definition'}, menuItems: [
-            {id: 'download-menu-item-without', condition: hasUpstream, menuItem: {clazz: 'fa', text: 'Without external services', action: 'downloadFlowWithoutExternalServices'}},
-            {id: 'download-menu-item-with', condition: hasDownstream, menuItem: {clazz: 'fa', text: 'With external services', action: 'downloadFlowWithExternalServices'}}
+            {id: 'download-menu-item-without', condition: supportsDownloadFlow, menuItem: {clazz: 'fa', text: 'Without external services', action: 'downloadFlowWithoutExternalServices'}},
+            {id: 'download-menu-item-with', condition: supportsDownloadFlow, menuItem: {clazz: 'fa', text: 'With external services', action: 'downloadFlowWithExternalServices'}}
         ]},
         {separator: true},
         {id: 'upload-template-menu-item', condition: canUploadTemplate, menuItem: {clazz: 'icon icon-template-import', text: 'Upload template', action: 'uploadTemplate'}},


[nifi] 14/16: NIFI-9993: Fixed bug in initialization in which the Content Repo did not properly increment the counter for how many files exist in the archive directories. This was causing the counter to become negative in some cases, which caused processors to incorrectly pause, waiting for content archive cleanup to occur when, in fact, there were no files archived

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit a213ad6c61b29dc9b82c0e476712452bcc370d51
Author: Mark Payne <ma...@hotmail.com>
AuthorDate: Fri May 6 10:11:10 2022 -0400

    NIFI-9993: Fixed bug in initialization in which the Content Repo did not properly increment the counter for how many files exist in the archive directories. This was causing the counter to become negative in some cases, which caused processors to incorrectly pause, waiting for content archive cleanup to occur when, in fact, there were no files archived
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    This closes #6021.
---
 .../repository/FileSystemRepository.java           | 35 +++++++++++++++++-----
 .../repository/TestFileSystemRepository.java       | 13 ++++++++
 2 files changed, 41 insertions(+), 7 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 52dad6e1fa..4bebb44bbe 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
@@ -357,8 +357,7 @@ public class FileSystemRepository implements ContentRepository {
                             }
 
                             // Check if this is an 'archive' directory
-                            final Path relativePath = realPath.relativize(file);
-                            if (relativePath.getNameCount() > 3 && ARCHIVE_DIR_NAME.equals(relativePath.subpath(1, 2).toString())) {
+                            if (isArchived(file)) {
                                 final long lastModifiedTime = getLastModTime(file);
 
                                 if (lastModifiedTime < oldestDateHolder.get()) {
@@ -401,6 +400,21 @@ public class FileSystemRepository implements ContentRepository {
         containers.putAll(realPathMap);
     }
 
+    // Visible for testing
+    boolean isArchived(final Path path) {
+        return isArchived(path.toFile());
+    }
+
+    // Visible for testing
+    boolean isArchived(final File file) {
+        final File parentFile = file.getParentFile();
+        if (parentFile == null) {
+            return false;
+        }
+
+        return ARCHIVE_DIR_NAME.equals(parentFile.getName());
+    }
+
     @Override
     public Set<String> getContainerNames() {
         return new HashSet<>(containerNames);
@@ -1194,7 +1208,7 @@ public class FileSystemRepository implements ContentRepository {
     // marked protected for visibility and ability to override for unit tests.
     protected boolean archive(final Path curPath) throws IOException {
         // check if already archived
-        final boolean alreadyArchived = ARCHIVE_DIR_NAME.equals(curPath.getParent().toFile().getName());
+        final boolean alreadyArchived = isArchived(curPath);
         if (alreadyArchived) {
             return false;
         }
@@ -1349,6 +1363,8 @@ public class FileSystemRepository implements ContentRepository {
         // Go through each container and grab the archived data into a List
         archiveExpirationLog.debug("Searching for more archived data to expire");
         final StopWatch stopWatch = new StopWatch(true);
+        final AtomicLong expiredFilesDeleted = new AtomicLong(0L);
+        final AtomicLong expiredBytesDeleted = new AtomicLong(0L);
         for (int i = 0; i < SECTIONS_PER_CONTAINER; i++) {
             final Path sectionContainer = container.resolve(String.valueOf(i));
             final Path archive = sectionContainer.resolve("archive");
@@ -1368,6 +1384,9 @@ public class FileSystemRepository implements ContentRepository {
                         final long lastModTime = getLastModTime(file);
                         if (lastModTime < timestampThreshold) {
                             try {
+                                expiredFilesDeleted.incrementAndGet();
+                                expiredBytesDeleted.addAndGet(file.toFile().length());
+
                                 Files.deleteIfExists(file);
                                 containerState.decrementArchiveCount();
                                 LOG.debug("Deleted archived ContentClaim with ID {} from Container {} because it was older than the configured max archival duration",
@@ -1443,7 +1462,7 @@ public class FileSystemRepository implements ContentRepository {
 
         // Remove the first 'counter' elements from the list because those were removed.
         notYetExceedingThreshold.subList(0, archiveFilesDeleted).clear();
-        LOG.info("Successfully deleted {} files ({}) from archive", archiveFilesDeleted, FormatUtils.formatDataSize(archiveBytesDeleted));
+        LOG.info("Successfully deleted {} files ({}) from archive", (archiveFilesDeleted + expiredFilesDeleted.get()), FormatUtils.formatDataSize(archiveBytesDeleted + expiredBytesDeleted.get()));
 
         final long deleteOldestMillis = stopWatch.getElapsed(TimeUnit.MILLISECONDS) - sortRemainingMillis - deleteExpiredMillis;
 
@@ -1708,7 +1727,7 @@ public class FileSystemRepository implements ContentRepository {
                 while (isWaitRequired()) {
                     try {
                         final String message = String.format("Unable to write flowfile content to content repository container %s due to archive file size constraints;" +
-                                " waiting for archive cleanup", containerName);
+                                " waiting for archive cleanup. Total number of files currently archived = %s", containerName, archivedFileCount.get());
                         LOG.warn(message);
                         eventReporter.reportEvent(Severity.WARNING, "FileSystemRepository", message);
                         condition.await();
@@ -1727,8 +1746,9 @@ public class FileSystemRepository implements ContentRepository {
 
             lock.lock();
             try {
+                long free = 0;
                 try {
-                    final long free = getContainerUsableSpace(containerName);
+                    free = getContainerUsableSpace(containerName);
                     bytesUsed = capacity - free;
                     checkUsedCutoffTimestamp = System.currentTimeMillis() + TimeUnit.MINUTES.toMillis(1L);
                 } catch (final Exception e) {
@@ -1737,7 +1757,8 @@ public class FileSystemRepository implements ContentRepository {
                     checkUsedCutoffTimestamp = 0L; // Signal that the free space should be calculated again next time it's checked.
                 }
 
-                LOG.debug("Container {} signaled to allow Content Claim Creation", containerName);
+                LOG.info("Archive cleanup completed for container {}; will now allow writing to this container. Bytes used = {}, bytes free = {}, capacity = {}", containerName,
+                    FormatUtils.formatDataSize(bytesUsed), FormatUtils.formatDataSize(free), FormatUtils.formatDataSize(capacity));
                 condition.signalAll();
             } finally {
                 lock.unlock();
diff --git a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
index 80a8f9d576..e05938ac93 100644
--- a/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
+++ b/nifi-nar-bundles/nifi-framework-bundle/nifi-framework/nifi-framework-core/src/test/java/org/apache/nifi/controller/repository/TestFileSystemRepository.java
@@ -46,6 +46,7 @@ import java.lang.reflect.Method;
 import java.nio.charset.StandardCharsets;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.nio.file.Paths;
 import java.nio.file.StandardCopyOption;
 import java.nio.file.StandardOpenOption;
 import java.text.NumberFormat;
@@ -127,6 +128,18 @@ public class TestFileSystemRepository {
                 + NumberFormat.getNumberInstance(Locale.US).format(bytesToWrite) + " bytes) for a write rate of " + mbps + " MB/s");
     }
 
+    @Test
+    public void testIsArchived() {
+        assertFalse(repository.isArchived(Paths.get("1.txt")));
+        assertFalse(repository.isArchived(Paths.get("a/1.txt")));
+        assertFalse(repository.isArchived(Paths.get("a/b/1.txt")));
+        assertFalse(repository.isArchived(Paths.get("a/archive/b/c/1.txt")));
+
+        assertTrue(repository.isArchived(Paths.get("archive/1.txt")));
+        assertTrue(repository.isArchived(Paths.get("a/archive/1.txt")));
+        assertTrue(repository.isArchived(Paths.get("a/b/c/archive/1.txt")));
+    }
+
     @Test
     public void testContentNotFoundExceptionThrownIfResourceClaimTooShort() throws IOException {
         final File contentFile = new File("target/content_repository/0/0.bin");


[nifi] 04/16: NIFI-9968 Added null check before System.setProperty() in test methods

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit f98d7299aabe030e99fee8919e22f4b1f783ca02
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed Apr 27 11:16:35 2022 -0500

    NIFI-9968 Added null check before System.setProperty() in test methods
    
    - Resolves build failures on Java 17 where the original user.timezone property returns null from System.getProperty()
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #6000
---
 .../src/test/java/org/apache/nifi/record/path/TestRecordPath.java     | 4 +++-
 .../java/org/apache/nifi/processors/standard/TestConvertRecord.java   | 4 +++-
 2 files changed, 6 insertions(+), 2 deletions(-)

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 100861fd2b..db3b13b786 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
@@ -79,7 +79,9 @@ public class TestRecordPath {
 
     @AfterAll
     public static void setSystemTimezone() {
-        System.setProperty(USER_TIMEZONE_PROPERTY, SYSTEM_TIMEZONE);
+        if (SYSTEM_TIMEZONE != null) {
+            System.setProperty(USER_TIMEZONE_PROPERTY, SYSTEM_TIMEZONE);
+        }
     }
 
     @Test
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
index a85e61ca81..47477b04ca 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestConvertRecord.java
@@ -405,7 +405,9 @@ public class TestConvertRecord {
             assertTrue(avroStream.hasNext());
             assertEquals(1, avroStream.next().get("date")); // see https://avro.apache.org/docs/1.10.0/spec.html#Date
         } finally {
-            System.setProperty("user.timezone", timezone);
+            if (timezone != null) {
+                System.setProperty("user.timezone", timezone);
+            }
         }
     }
 }


[nifi] 16/16: NIFI-10005

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 9d0e92eb345ac98e9876e7a453eaa973d7be8895
Author: Joe Witt <jo...@apache.org>
AuthorDate: Mon May 9 10:58:16 2022 -0700

    NIFI-10005
---
 nifi-docker/dockerhub/DockerImage.txt | 2 +-
 nifi-docker/dockerhub/Dockerfile      | 2 +-
 2 files changed, 2 insertions(+), 2 deletions(-)

diff --git a/nifi-docker/dockerhub/DockerImage.txt b/nifi-docker/dockerhub/DockerImage.txt
index 9dda7c99fc..c7ce11748b 100644
--- a/nifi-docker/dockerhub/DockerImage.txt
+++ b/nifi-docker/dockerhub/DockerImage.txt
@@ -13,4 +13,4 @@
 # See the License for the specific language governing permissions and
 # limitations under the License.
 
-apache/nifi:1.16.1
+apache/nifi:1.16.2
diff --git a/nifi-docker/dockerhub/Dockerfile b/nifi-docker/dockerhub/Dockerfile
index f04ab9e70c..765743c145 100644
--- a/nifi-docker/dockerhub/Dockerfile
+++ b/nifi-docker/dockerhub/Dockerfile
@@ -24,7 +24,7 @@ LABEL site="https://nifi.apache.org"
 
 ARG UID=1000
 ARG GID=1000
-ARG NIFI_VERSION=1.16.1
+ARG NIFI_VERSION=1.16.2
 ARG BASE_URL=https://archive.apache.org/dist
 ARG MIRROR_BASE_URL=${MIRROR_BASE_URL:-${BASE_URL}}
 ARG DISTRO_PATH=${DISTRO_PATH:-${NIFI_VERSION}}


[nifi] 06/16: NIFI-9977 In StandardOauth2AccessTokenProvider add new property to be able to set "scope".

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 25031dd8192e81d0157e2b6aed18f26362599dd7
Author: Tamas Palfy <ta...@gmail.com>
AuthorDate: Fri Apr 29 17:18:29 2022 +0200

    NIFI-9977 In StandardOauth2AccessTokenProvider add new property to be able to set "scope".
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #6006.
---
 .../oauth2/StandardOauth2AccessTokenProvider.java     | 19 +++++++++++++++++++
 1 file changed, 19 insertions(+)

diff --git a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
index 48e47ff19a..c09ecc9e96 100644
--- a/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
+++ b/nifi-nar-bundles/nifi-standard-services/nifi-oauth2-provider-bundle/nifi-oauth2-provider-service/src/main/java/org/apache/nifi/oauth2/StandardOauth2AccessTokenProvider.java
@@ -122,6 +122,14 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
         .addValidator(StandardValidators.NON_BLANK_VALIDATOR)
         .build();
 
+    public static final PropertyDescriptor SCOPE = new PropertyDescriptor.Builder()
+        .name("scope")
+        .displayName("Scope")
+        .description("Space-delimited, case-sensitive list of scopes of the access request (as per the OAuth 2.0 specification)")
+        .required(false)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .build();
+
     public static final PropertyDescriptor REFRESH_WINDOW = new PropertyDescriptor.Builder()
             .name("refresh-window")
             .displayName("Refresh Window")
@@ -146,6 +154,7 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
         PASSWORD,
         CLIENT_ID,
         CLIENT_SECRET,
+        SCOPE,
         REFRESH_WINDOW,
         SSL_CONTEXT
     ));
@@ -162,6 +171,7 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
     private volatile String password;
     private volatile String clientId;
     private volatile String clientSecret;
+    private volatile String scope;
     private volatile long refreshWindowSeconds;
 
     private volatile AccessToken accessDetails;
@@ -182,6 +192,7 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
         password = context.getProperty(PASSWORD).getValue();
         clientId = context.getProperty(CLIENT_ID).evaluateAttributeExpressions().getValue();
         clientSecret = context.getProperty(CLIENT_SECRET).getValue();
+        scope = context.getProperty(SCOPE).getValue();
 
         refreshWindowSeconds = context.getProperty(REFRESH_WINDOW).asTimePeriod(TimeUnit.SECONDS);
     }
@@ -264,6 +275,10 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
             acquireTokenBuilder.add("client_secret", clientSecret);
         }
 
+        if (scope != null) {
+            acquireTokenBuilder.add("scope", scope);
+        }
+
         RequestBody acquireTokenRequestBody = acquireTokenBuilder.build();
 
         Request acquireTokenRequest = new Request.Builder()
@@ -286,6 +301,10 @@ public class StandardOauth2AccessTokenProvider extends AbstractControllerService
             refreshTokenBuilder.add("client_secret", clientSecret);
         }
 
+        if (scope != null) {
+            refreshTokenBuilder.add("scope", scope);
+        }
+
         RequestBody refreshTokenRequestBody = refreshTokenBuilder.build();
 
         Request refreshRequest = new Request.Builder()


[nifi] 10/16: NIFI-9980 Corrected conflicting GCP dependencies

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 0f6d7c1e7d3d2487646bc77ba45662f88afecef2
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Mon May 2 11:51:39 2022 -0500

    NIFI-9980 Corrected conflicting GCP dependencies
    
    - Replaced google-cloud-bom 0.172.0 with libraries-bom 25.2.0 in nifi-gcp-bundle
    - Removed specific versions from Google dependencies in nifi-gcp-processors and nifi-gcp-services-api
    
    Signed-off-by: Pierre Villard <pi...@gmail.com>
    
    This closes #6010.
---
 nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml |  2 --
 .../nifi-gcp-bundle/nifi-gcp-services-api/pom.xml            | 12 ------------
 nifi-nar-bundles/nifi-gcp-bundle/pom.xml                     | 12 +++---------
 3 files changed, 3 insertions(+), 23 deletions(-)

diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
index 92e153afa9..f5cb3983cd 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-processors/pom.xml
@@ -73,7 +73,6 @@
         <dependency>
             <groupId>com.google.cloud</groupId>
             <artifactId>google-cloud-core</artifactId>
-            <version>2.1.7</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.google.code.findbugs</groupId>
@@ -108,7 +107,6 @@
         <dependency>
             <groupId>com.google.cloud</groupId>
             <artifactId>google-cloud-pubsublite</artifactId>
-            <version>1.3.0</version>
             <exclusions>
                 <exclusion>
                     <groupId>commons-logging</groupId>
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
index e5097ae057..250d51f5f9 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/nifi-gcp-services-api/pom.xml
@@ -25,17 +25,6 @@
     <artifactId>nifi-gcp-services-api</artifactId>
     <packaging>jar</packaging>
 
-    <dependencyManagement>
-        <dependencies>
-            <!-- Override Guava version from google-auth-library-oauth2-http -->
-            <dependency>
-                <groupId>com.google.guava</groupId>
-                <artifactId>guava</artifactId>
-                <version>31.0.1-jre</version>
-            </dependency>
-        </dependencies>
-    </dependencyManagement>
-
     <dependencies>
         <dependency>
             <groupId>org.apache.nifi</groupId>
@@ -44,7 +33,6 @@
         <dependency>
             <groupId>com.google.auth</groupId>
             <artifactId>google-auth-library-oauth2-http</artifactId>
-            <version>1.2.1</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.google.code.findbugs</groupId>
diff --git a/nifi-nar-bundles/nifi-gcp-bundle/pom.xml b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
index 681d944381..85b3e6e133 100644
--- a/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
+++ b/nifi-nar-bundles/nifi-gcp-bundle/pom.xml
@@ -27,24 +27,18 @@
     <packaging>pom</packaging>
 
     <properties>
-        <google.cloud.sdk.version>0.172.0</google.cloud.sdk.version>
+        <google.libraries.version>25.2.0</google.libraries.version>
     </properties>
 
     <dependencyManagement>
         <dependencies>
             <dependency>
-            	<!-- https://github.com/googleapis/java-cloud-bom -->
                 <groupId>com.google.cloud</groupId>
-                <artifactId>google-cloud-bom</artifactId>
-                <version>${google.cloud.sdk.version}</version>
+                <artifactId>libraries-bom</artifactId>
+                <version>${google.libraries.version}</version>
                 <type>pom</type>
                 <scope>import</scope>
             </dependency>
-            <dependency>
-                <groupId>com.google.guava</groupId>
-                <artifactId>guava</artifactId>
-                <version>31.0.1-jre</version>
-            </dependency>
         </dependencies>
     </dependencyManagement>
 


[nifi] 02/16: NIFI-9944: Fixed issue with using modules in InvokeScriptedProcessor

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 008ecfafb5048c68f93fd8f12decc9a0706a7414
Author: Matthew Burgess <ma...@apache.org>
AuthorDate: Mon Apr 25 16:15:23 2022 -0400

    NIFI-9944: Fixed issue with using modules in InvokeScriptedProcessor
---
 .../processors/script/InvokeScriptedProcessor.java    |   4 ----
 .../org/apache/nifi/script/ScriptRunnerFactory.java   |   2 +-
 .../nifi/processors/script/TestInvokeJavascript.java  |   8 ++++----
 .../src/test/resources/jar/commons-math3-3.2.jar      | Bin 0 -> 1692782 bytes
 .../src/test/resources/javascript/test_reader.js      |   4 +++-
 5 files changed, 8 insertions(+), 10 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 93a9984c83..8c89233d88 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
@@ -262,10 +262,6 @@ public class InvokeScriptedProcessor extends AbstractSessionFactoryProcessor {
 
             scriptNeedsReload.set(true);
             scriptRunner = null; //reset engine. This happens only when a processor is stopped, so there won't be any performance impact in run-time.
-            if (isConfigurationRestored()) {
-                // Once the configuration has been restored, each call to onPropertyModified() is due to a change made after the processor was loaded, so reload the script
-                setup();
-            }
         } else if (instance != null) {
             // If the script provides a Processor, call its onPropertyModified() method
             try {
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptRunnerFactory.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptRunnerFactory.java
index 1b90851a67..6bd2fe6aa9 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptRunnerFactory.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/main/java/org/apache/nifi/script/ScriptRunnerFactory.java
@@ -79,7 +79,7 @@ public class ScriptRunnerFactory {
 
         if (!"Clojure".equals(scriptEngineName)
                 && !"Groovy".equals(scriptEngineName)
-                && "ECMAScript".equals(scriptEngineName)) {
+                && !"ECMAScript".equals(scriptEngineName)) {
             return new URL[0];
         }
 
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
index 8c4eda2cb5..c56612bd2a 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/java/org/apache/nifi/processors/script/TestInvokeJavascript.java
@@ -58,7 +58,7 @@ public class TestInvokeJavascript extends BaseScriptTest {
     public void testReadFlowFileContentAndStoreInFlowFileAttribute() {
         runner.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
         runner.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        runner.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/javascript");
+        runner.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
 
         runner.assertValid();
         runner.enqueue("test content".getBytes(StandardCharsets.UTF_8));
@@ -86,7 +86,7 @@ public class TestInvokeJavascript extends BaseScriptTest {
 
         context.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
         context.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/javascript");
+        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
         // State Manger is unused, and a null reference is specified
         processor.customValidate(new MockValidationContext(context));
         processor.setup(context);
@@ -112,7 +112,7 @@ public class TestInvokeJavascript extends BaseScriptTest {
      * @Any error encountered while testing
      */
     @Test
-    public void testScriptDefinedRelationship() {
+    public void testScriptDefinedRelationshipWithExternalJar() {
         InvokeScriptedProcessor processor = new InvokeScriptedProcessor();
         MockProcessContext context = new MockProcessContext(processor);
         MockProcessorInitializationContext initContext = new MockProcessorInitializationContext(processor, context);
@@ -121,7 +121,7 @@ public class TestInvokeJavascript extends BaseScriptTest {
 
         context.setProperty(scriptingComponent.getScriptingComponentHelper().SCRIPT_ENGINE, "ECMAScript");
         context.setProperty(ScriptingComponentUtils.SCRIPT_FILE, "target/test/resources/javascript/test_reader.js");
-        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/javascript");
+        context.setProperty(ScriptingComponentUtils.MODULES, "target/test/resources/jar");
 
         // State Manger is unused, and a null reference is specified
         processor.customValidate(new MockValidationContext(context));
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar
new file mode 100644
index 0000000000..f8b7db295b
Binary files /dev/null and b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/jar/commons-math3-3.2.jar differ
diff --git a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_reader.js b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_reader.js
index 44d9530d31..f82a31a569 100644
--- a/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_reader.js
+++ b/nifi-nar-bundles/nifi-scripting-bundle/nifi-scripting-processors/src/test/resources/javascript/test_reader.js
@@ -27,6 +27,7 @@ var Relationship = Java.type("org.apache.nifi.processor.Relationship");
 var PropertyDescriptor = Java.type("org.apache.nifi.components.PropertyDescriptor");
 var StandardValidators = Java.type("org.apache.nifi.processor.util.StandardValidators");
 var StreamUtils = Java.type("org.apache.nifi.stream.io.StreamUtils");
+var Primes = Java.type("org.apache.commons.math3.primes.Primes")
 
 var REL_TEST = new Relationship.Builder()
         .name("test")
@@ -72,7 +73,8 @@ var processor = new Object() {
 
         this.logger.info("Read content={}", [content]);
 
-        flowFile = session.putAttribute(flowFile, "from-content", content)
+        flowFile = session.putAttribute(flowFile, "from-content", content);
+        flowFile = session.putAttribute(flowFile, "is3Prime", Primes.isPrime(3))
         // transfer
         session.transfer(flowFile, REL_TEST)
         session.commitAsync()


[nifi] 11/16: NIFI-9932 Upgrade minimist from 1.2.5 to 1.2.6

Posted by jo...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

joewitt pushed a commit to branch support/nifi-1.16
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 010e0bc75dd20771db8d5efa9901a3e9238c788b
Author: dependabot[bot] <49...@users.noreply.github.com>
AuthorDate: Mon Apr 18 10:04:36 2022 +0000

    NIFI-9932 Upgrade minimist from 1.2.5 to 1.2.6
    
    Bumps [minimist](https://github.com/substack/minimist) from 1.2.5 to 1.2.6.
    - [Release notes](https://github.com/substack/minimist/releases)
    - [Commits](https://github.com/substack/minimist/compare/1.2.5...1.2.6)
    
    ---
    updated-dependencies:
    - dependency-name: minimist
      dependency-type: indirect
    ...
    
    This closes #5971
    
    Signed-off-by: David Handermann <ex...@apache.org>
---
 .../src/main/package-lock.json                     | 116 ++-------------------
 1 file changed, 6 insertions(+), 110 deletions(-)

diff --git a/nifi-registry/nifi-registry-core/nifi-registry-web-ui/src/main/package-lock.json b/nifi-registry/nifi-registry-core/nifi-registry-web-ui/src/main/package-lock.json
index ba8b4f169a..3668cbab78 100644
--- a/nifi-registry/nifi-registry-core/nifi-registry-web-ui/src/main/package-lock.json
+++ b/nifi-registry/nifi-registry-core/nifi-registry-web-ui/src/main/package-lock.json
@@ -3731,18 +3731,6 @@
                 "fsevents": "~2.3.2"
             }
         },
-        "node_modules/chokidar/node_modules/glob-parent": {
-            "version": "6.0.2",
-            "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-            "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-            "dev": true,
-            "dependencies": {
-                "is-glob": "^4.0.1"
-            },
-            "engines": {
-                "node": ">= 6"
-            }
-        },
         "node_modules/chownr": {
             "version": "2.0.0",
             "resolved": "https://registry.npmjs.org/chownr/-/chownr-2.0.0.tgz",
@@ -6348,18 +6336,6 @@
                 "node": ">=8.6.0"
             }
         },
-        "node_modules/fast-glob/node_modules/glob-parent": {
-            "version": "6.0.2",
-            "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-            "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-            "dev": true,
-            "dependencies": {
-                "is-glob": "^4.0.1"
-            },
-            "engines": {
-                "node": ">= 6"
-            }
-        },
         "node_modules/fast-json-stable-stringify": {
             "version": "2.1.0",
             "resolved": "https://registry.npmjs.org/fast-json-stable-stringify/-/fast-json-stable-stringify-2.1.0.tgz",
@@ -9604,9 +9580,9 @@
             }
         },
         "node_modules/minimist": {
-            "version": "1.2.5",
-            "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.5.tgz",
-            "integrity": "sha512-FM9nNUYrRBAELZQT3xeZQ7fmMOBg6nWNmJKTcgsJeaLstP/UODVpGsr5OhXhhXg6f+qtJ8uiZ+PUxkDWcgIXLw==",
+            "version": "1.2.6",
+            "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.6.tgz",
+            "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==",
             "dev": true
         },
         "node_modules/minimist-options": {
@@ -10577,13 +10553,6 @@
             "integrity": "sha512-BapA40NHICOS+USX9SN4tyhq+A2RrN/Ws5F0Z5aMHDp98Fl86lX8Oti8B7uN93L4Ifv4fHOEA+pQw87gmMO/lQ==",
             "dev": true
         },
-        "node_modules/path-dirname": {
-            "version": "1.0.2",
-            "resolved": "https://registry.npmjs.org/path-dirname/-/path-dirname-1.0.2.tgz",
-            "integrity": "sha1-zDPSTVJeCZpTiMAzbG4yuRYGCeA=",
-            "dev": true,
-            "optional": true
-        },
         "node_modules/path-exists": {
             "version": "3.0.0",
             "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz",
@@ -14741,30 +14710,6 @@
                 "node": ">= 4.0"
             }
         },
-        "node_modules/watchpack-chokidar2/node_modules/glob-parent": {
-            "version": "6.0.2",
-            "resolved": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-            "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-            "dev": true,
-            "optional": true,
-            "dependencies": {
-                "is-glob": "^3.1.0",
-                "path-dirname": "^1.0.0"
-            }
-        },
-        "node_modules/watchpack-chokidar2/node_modules/glob-parent/node_modules/is-glob": {
-            "version": "3.1.0",
-            "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-3.1.0.tgz",
-            "integrity": "sha1-e6WuJCF4BKxwcHuWkiVnSGzD6Eo=",
-            "dev": true,
-            "optional": true,
-            "dependencies": {
-                "is-extglob": "^2.1.0"
-            },
-            "engines": {
-                "node": ">=0.10.0"
-            }
-        },
         "node_modules/watchpack-chokidar2/node_modules/is-binary-path": {
             "version": "1.0.1",
             "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-1.0.1.tgz",
@@ -18509,16 +18454,6 @@
                 "is-glob": "~4.0.1",
                 "normalize-path": "~3.0.0",
                 "readdirp": "~3.6.0"
-            },
-            "dependencies": {
-                "glob-parent": {
-                    "version": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-                    "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-                    "dev": true,
-                    "requires": {
-                        "is-glob": "^4.0.1"
-                    }
-                }
             }
         },
         "chownr": {
@@ -20591,16 +20526,6 @@
                 "glob-parent": "^5.1.2",
                 "merge2": "^1.3.0",
                 "micromatch": "^4.0.4"
-            },
-            "dependencies": {
-                "glob-parent": {
-                    "version": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-                    "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-                    "dev": true,
-                    "requires": {
-                        "is-glob": "^4.0.1"
-                    }
-                }
             }
         },
         "fast-json-stable-stringify": {
@@ -23074,9 +22999,9 @@
             }
         },
         "minimist": {
-            "version": "1.2.5",
-            "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.5.tgz",
-            "integrity": "sha512-FM9nNUYrRBAELZQT3xeZQ7fmMOBg6nWNmJKTcgsJeaLstP/UODVpGsr5OhXhhXg6f+qtJ8uiZ+PUxkDWcgIXLw==",
+            "version": "1.2.6",
+            "resolved": "https://registry.npmjs.org/minimist/-/minimist-1.2.6.tgz",
+            "integrity": "sha512-Jsjnk4bw3YJqYzbdyBiNsPWHPfO++UGG749Cxs6peCu5Xg4nrena6OVxOYxrQTqww0Jmwt+Ref8rggumkTLz9Q==",
             "dev": true
         },
         "minimist-options": {
@@ -23838,13 +23763,6 @@
             "integrity": "sha512-BapA40NHICOS+USX9SN4tyhq+A2RrN/Ws5F0Z5aMHDp98Fl86lX8Oti8B7uN93L4Ifv4fHOEA+pQw87gmMO/lQ==",
             "dev": true
         },
-        "path-dirname": {
-            "version": "1.0.2",
-            "resolved": "https://registry.npmjs.org/path-dirname/-/path-dirname-1.0.2.tgz",
-            "integrity": "sha1-zDPSTVJeCZpTiMAzbG4yuRYGCeA=",
-            "dev": true,
-            "optional": true
-        },
         "path-exists": {
             "version": "3.0.0",
             "resolved": "https://registry.npmjs.org/path-exists/-/path-exists-3.0.0.tgz",
@@ -27128,28 +27046,6 @@
                         "nan": "^2.12.1"
                     }
                 },
-                "glob-parent": {
-                    "version": "https://registry.npmjs.org/glob-parent/-/glob-parent-6.0.2.tgz",
-                    "integrity": "sha512-XxwI8EOhVQgWp6iDL+3b0r86f4d6AX6zSU55HfB4ydCEuXLXc5FcYeOu+nnGftS4TEju/11rt4KJPTMgbfmv4A==",
-                    "dev": true,
-                    "optional": true,
-                    "requires": {
-                        "is-glob": "^3.1.0",
-                        "path-dirname": "^1.0.0"
-                    },
-                    "dependencies": {
-                        "is-glob": {
-                            "version": "3.1.0",
-                            "resolved": "https://registry.npmjs.org/is-glob/-/is-glob-3.1.0.tgz",
-                            "integrity": "sha1-e6WuJCF4BKxwcHuWkiVnSGzD6Eo=",
-                            "dev": true,
-                            "optional": true,
-                            "requires": {
-                                "is-extglob": "^2.1.0"
-                            }
-                        }
-                    }
-                },
                 "is-binary-path": {
                     "version": "1.0.1",
                     "resolved": "https://registry.npmjs.org/is-binary-path/-/is-binary-path-1.0.1.tgz",