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/01/12 20:01:22 UTC

[nifi] branch support/nifi-1.15 updated (91b50ca -> b5f34de)

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

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


    from 91b50ca  NIFI-9534 Upgraded Log4j 2 BOM from 2.17.0 to 2.17.1
     new 87556a7  NIFI-7835 Added authenticated SOCKS proxy support for SFTP
     new 8f50e90  NIFI-8605 Adding a new property for ExecuteSQL and ExecuteSQLRecord processors to enable/disable auto committing change the default value of auto commit function to true Changed the auto commit property name and add more details in the description If the auto commit is set to false, commit() is called for consistency adds unit tests Fix the check style issue of having more than 200 characters in single line
     new c71a18f  NIFI-9524 exclude commons-logging and log4j-core banned dependencies for other build profiles as well (#5625)
     new b5f34de  NIFI-9564 This closes #5652. Removed unnecessary logback-classic test dependencies

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 nifi-bootstrap/pom.xml                             |   5 -
 nifi-commons/nifi-bootstrap-utils/pom.xml          |   6 -
 .../nifi-hadoop-libraries-nar/pom.xml              |  44 ++-
 .../nifi-hive-bundle/nifi-hive3-nar/pom.xml        |  24 ++
 .../nifi-ranger-bundle/nifi-ranger-plugin/pom.xml  |  16 +
 .../nifi-standard-processors/pom.xml               |   5 +
 .../processors/standard/AbstractExecuteSQL.java    | 339 +++++++++++----------
 .../nifi/processors/standard/ExecuteSQL.java       |   1 +
 .../nifi/processors/standard/ExecuteSQLRecord.java |   1 +
 .../{ssh => socket}/ProxySocketFactory.java        |   2 +-
 .../standard/socket/SocketFactoryProvider.java     |  20 +-
 .../socket/StandardSocketFactoryProvider.java      |  65 ++++
 .../standard/ssh/StandardSSHClientProvider.java    |   7 +-
 .../nifi/processors/standard/TestExecuteSQL.java   |  17 ++
 .../processors/standard/TestExecuteSQLRecord.java  |  17 ++
 .../{ssh => socket}/ProxySocketFactoryTest.java    |   2 +-
 .../socket/StandardSocketFactoryProviderTest.java  |  80 +++++
 .../nifi-registry-ranger-plugin/pom.xml            |  16 +
 18 files changed, 474 insertions(+), 193 deletions(-)
 rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/{ssh => socket}/ProxySocketFactory.java (98%)
 copy nifi-framework-api/src/main/java/org/apache/nifi/authorization/AuthorizationAuditor.java => nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/SocketFactoryProvider.java (65%)
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProvider.java
 rename nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/{ssh => socket}/ProxySocketFactoryTest.java (96%)
 create mode 100644 nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProviderTest.java

[nifi] 04/04: NIFI-9564 This closes #5652. Removed unnecessary logback-classic test 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.15
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit b5f34de4d78890f432157e51052e0ef4d700f8d6
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Wed Jan 12 10:48:56 2022 -0600

    NIFI-9564 This closes #5652. Removed unnecessary logback-classic test dependencies
    
    Signed-off-by: Joe Witt <jo...@apache.org>
---
 nifi-bootstrap/pom.xml                    | 5 -----
 nifi-commons/nifi-bootstrap-utils/pom.xml | 6 ------
 2 files changed, 11 deletions(-)

diff --git a/nifi-bootstrap/pom.xml b/nifi-bootstrap/pom.xml
index 4c5ca71..5577489 100644
--- a/nifi-bootstrap/pom.xml
+++ b/nifi-bootstrap/pom.xml
@@ -85,10 +85,5 @@ language governing permissions and limitations under the License. -->
             <artifactId>nifi-properties-loader</artifactId>
             <version>1.15.3-SNAPSHOT</version>
         </dependency>
-        <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 </project>
diff --git a/nifi-commons/nifi-bootstrap-utils/pom.xml b/nifi-commons/nifi-bootstrap-utils/pom.xml
index 51abea5..4ca45fb 100644
--- a/nifi-commons/nifi-bootstrap-utils/pom.xml
+++ b/nifi-commons/nifi-bootstrap-utils/pom.xml
@@ -31,11 +31,5 @@ language governing permissions and limitations under the License. -->
             <artifactId>jna-platform</artifactId>
             <version>4.4.0</version>
         </dependency>
-
-        <dependency>
-            <groupId>ch.qos.logback</groupId>
-            <artifactId>logback-classic</artifactId>
-            <scope>test</scope>
-        </dependency>
     </dependencies>
 </project>

[nifi] 01/04: NIFI-7835 Added authenticated SOCKS proxy support for SFTP

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.15
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 87556a79c57340b00e95c6215e5a8e7e7490c2e6
Author: exceptionfactory <ex...@apache.org>
AuthorDate: Tue Nov 16 15:22:09 2021 -0600

    NIFI-7835 Added authenticated SOCKS proxy support for SFTP
    
    NIFI-7749 Added authenticated HTTP proxy support for SFTP
    
    - Added StandardSocketFactoryProvider to return SocketFactory based on credentials
    
    Signed-off-by: Joe Gresock <jg...@gmail.com>
    
    This closes #5624.
---
 .../nifi-standard-processors/pom.xml               |  5 ++
 .../{ssh => socket}/ProxySocketFactory.java        |  2 +-
 .../standard/socket/SocketFactoryProvider.java     | 34 +++++++++
 .../socket/StandardSocketFactoryProvider.java      | 65 ++++++++++++++++++
 .../standard/ssh/StandardSSHClientProvider.java    |  7 +-
 .../{ssh => socket}/ProxySocketFactoryTest.java    |  2 +-
 .../socket/StandardSocketFactoryProviderTest.java  | 80 ++++++++++++++++++++++
 7 files changed, 191 insertions(+), 4 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
index 0c61dcb..18859c5 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/pom.xml
@@ -142,6 +142,11 @@
             <artifactId>sshj</artifactId>
         </dependency>
         <dependency>
+            <groupId>com.exceptionfactory.socketbroker</groupId>
+            <artifactId>socketbroker</artifactId>
+            <version>1.0.2</version>
+        </dependency>
+        <dependency>
             <groupId>org.eclipse.jetty</groupId>
             <artifactId>jetty-server</artifactId>
         </dependency>
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactory.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/ProxySocketFactory.java
similarity index 98%
rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactory.java
rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/ProxySocketFactory.java
index 4414880..d5125b0 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactory.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/ProxySocketFactory.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.processors.standard.ssh;
+package org.apache.nifi.processors.standard.socket;
 
 import javax.net.SocketFactory;
 import java.io.IOException;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/SocketFactoryProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/SocketFactoryProvider.java
new file mode 100644
index 0000000..357eb47
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/SocketFactoryProvider.java
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard.socket;
+
+import org.apache.nifi.proxy.ProxyConfiguration;
+
+import javax.net.SocketFactory;
+
+/**
+ * Socket Factory Provider abstracts implementation selection based on Proxy Configuration
+ */
+public interface SocketFactoryProvider {
+    /**
+     * Get Socket Factory based on provided Proxy Configuration
+     *
+     * @param proxyConfiguration Proxy Configuration required
+     * @return Socket Factory
+     */
+    SocketFactory getSocketFactory(ProxyConfiguration proxyConfiguration);
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProvider.java
new file mode 100644
index 0000000..967943d
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProvider.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard.socket;
+
+import com.exceptionfactory.socketbroker.BrokeredSocketFactory;
+import com.exceptionfactory.socketbroker.configuration.AuthenticationCredentials;
+import com.exceptionfactory.socketbroker.configuration.BrokerConfiguration;
+import com.exceptionfactory.socketbroker.configuration.ProxyType;
+import com.exceptionfactory.socketbroker.configuration.StandardBrokerConfiguration;
+import com.exceptionfactory.socketbroker.configuration.StandardUsernamePasswordAuthenticationCredentials;
+import org.apache.nifi.proxy.ProxyConfiguration;
+
+import javax.net.SocketFactory;
+import java.net.InetSocketAddress;
+import java.net.Proxy;
+import java.util.Objects;
+
+/**
+ * Standard implementation of Socket Factory Provider support authenticated or unauthenticated SOCKS or HTTP proxies
+ */
+public class StandardSocketFactoryProvider implements SocketFactoryProvider {
+    /**
+     * Get Socket Factory returns ProxySocketFactory without credentials and BrokeredSocketFactory with credentials
+     *
+     * @param proxyConfiguration Proxy Configuration required
+     * @return Socket Factory
+     */
+    @Override
+    public SocketFactory getSocketFactory(final ProxyConfiguration proxyConfiguration) {
+        Objects.requireNonNull(proxyConfiguration, "Proxy Configuration required");
+
+        final String userName = proxyConfiguration.getProxyUserName();
+        final SocketFactory socketFactory;
+        if (userName == null) {
+            final Proxy proxy = proxyConfiguration.createProxy();
+            socketFactory = new ProxySocketFactory(proxy);
+        } else {
+            final Proxy.Type proxyType = proxyConfiguration.getProxyType();
+            final ProxyType brokerProxyType = Proxy.Type.SOCKS == proxyType ? ProxyType.SOCKS5 : ProxyType.HTTP_CONNECT;
+            final InetSocketAddress proxySocketAddress = new InetSocketAddress(proxyConfiguration.getProxyServerHost(), proxyConfiguration.getProxyServerPort());
+
+            final String proxyPassword = proxyConfiguration.getProxyUserPassword();
+            final char[] brokerProxyPassword = proxyPassword == null ? new char[]{} : proxyPassword.toCharArray();
+            final AuthenticationCredentials credentials = new StandardUsernamePasswordAuthenticationCredentials(userName, brokerProxyPassword);
+
+            final BrokerConfiguration brokerConfiguration = new StandardBrokerConfiguration(brokerProxyType, proxySocketAddress, credentials);
+            socketFactory = new BrokeredSocketFactory(brokerConfiguration, SocketFactory.getDefault());
+        }
+        return socketFactory;
+    }
+}
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/StandardSSHClientProvider.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/StandardSSHClientProvider.java
index 758ba06..f16da8d 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/StandardSSHClientProvider.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ssh/StandardSSHClientProvider.java
@@ -34,6 +34,8 @@ import net.schmizz.sshj.userauth.password.PasswordFinder;
 import net.schmizz.sshj.userauth.password.PasswordUtils;
 
 import org.apache.nifi.context.PropertyContext;
+import org.apache.nifi.processors.standard.socket.SocketFactoryProvider;
+import org.apache.nifi.processors.standard.socket.StandardSocketFactoryProvider;
 import org.apache.nifi.proxy.ProxyConfiguration;
 import org.apache.nifi.util.StringUtils;
 
@@ -68,6 +70,8 @@ import static org.apache.nifi.processors.standard.util.SFTPTransfer.USE_COMPRESS
 public class StandardSSHClientProvider implements SSHClientProvider {
     private static final SSHConfigProvider SSH_CONFIG_PROVIDER = new StandardSSHConfigProvider();
 
+    private static final SocketFactoryProvider SOCKET_FACTORY_PROVIDER = new StandardSocketFactoryProvider();
+
     private static final List<Proxy.Type> SUPPORTED_PROXY_TYPES = Arrays.asList(Proxy.Type.HTTP, Proxy.Type.SOCKS);
 
     private static final String ADDRESS_FORMAT = "%s:%d";
@@ -170,8 +174,7 @@ public class StandardSSHClientProvider implements SSHClientProvider {
         final ProxyConfiguration proxyConfiguration = ProxyConfiguration.getConfiguration(context, createComponentProxyConfigSupplier(context));
         final Proxy.Type proxyType = proxyConfiguration.getProxyType();
         if (SUPPORTED_PROXY_TYPES.contains(proxyType)) {
-            final Proxy proxy = proxyConfiguration.createProxy();
-            final SocketFactory socketFactory = new ProxySocketFactory(proxy);
+            final SocketFactory socketFactory = SOCKET_FACTORY_PROVIDER.getSocketFactory(proxyConfiguration);
             client.setSocketFactory(socketFactory);
         }
     }
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactoryTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/ProxySocketFactoryTest.java
similarity index 96%
rename from nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactoryTest.java
rename to nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/ProxySocketFactoryTest.java
index d599e18..90a5e27 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/ssh/ProxySocketFactoryTest.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/ProxySocketFactoryTest.java
@@ -14,7 +14,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.nifi.processors.standard.ssh;
+package org.apache.nifi.processors.standard.socket;
 
 import org.junit.jupiter.api.Test;
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProviderTest.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProviderTest.java
new file mode 100644
index 0000000..7680f13
--- /dev/null
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/socket/StandardSocketFactoryProviderTest.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.nifi.processors.standard.socket;
+
+import com.exceptionfactory.socketbroker.BrokeredSocketFactory;
+import org.apache.nifi.proxy.ProxyConfiguration;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import javax.net.SocketFactory;
+import java.net.Proxy;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+public class StandardSocketFactoryProviderTest {
+    private static final String HOST = "localhost";
+
+    private static final int PORT = 1080;
+
+    private static final String USERNAME = "user";
+
+    private static final String PASSWORD = "password";
+
+    private StandardSocketFactoryProvider provider;
+
+    @BeforeEach
+    public void setProvider() {
+        provider = new StandardSocketFactoryProvider();
+    }
+
+    @Test
+    public void testGetSocketFactoryWithoutCredentials() {
+        final ProxyConfiguration proxyConfiguration = new ProxyConfiguration();
+        proxyConfiguration.setProxyType(Proxy.Type.SOCKS);
+        proxyConfiguration.setProxyServerHost(HOST);
+        proxyConfiguration.setProxyServerPort(PORT);
+
+        final SocketFactory socketFactory = provider.getSocketFactory(proxyConfiguration);
+        assertEquals(ProxySocketFactory.class, socketFactory.getClass());
+    }
+
+    @Test
+    public void testGetSocketFactoryWithUsername() {
+        final ProxyConfiguration proxyConfiguration = new ProxyConfiguration();
+        proxyConfiguration.setProxyType(Proxy.Type.SOCKS);
+        proxyConfiguration.setProxyServerHost(HOST);
+        proxyConfiguration.setProxyServerPort(PORT);
+        proxyConfiguration.setProxyUserName(USERNAME);
+
+        final SocketFactory socketFactory = provider.getSocketFactory(proxyConfiguration);
+        assertEquals(BrokeredSocketFactory.class, socketFactory.getClass());
+    }
+
+    @Test
+    public void testGetSocketFactoryWithUsernamePassword() {
+        final ProxyConfiguration proxyConfiguration = new ProxyConfiguration();
+        proxyConfiguration.setProxyType(Proxy.Type.SOCKS);
+        proxyConfiguration.setProxyServerHost(HOST);
+        proxyConfiguration.setProxyServerPort(PORT);
+        proxyConfiguration.setProxyUserName(USERNAME);
+        proxyConfiguration.setProxyUserPassword(PASSWORD);
+
+        final SocketFactory socketFactory = provider.getSocketFactory(proxyConfiguration);
+        assertEquals(BrokeredSocketFactory.class, socketFactory.getClass());
+    }
+}

[nifi] 02/04: NIFI-8605 Adding a new property for ExecuteSQL and ExecuteSQLRecord processors to enable/disable auto committing change the default value of auto commit function to true Changed the auto commit property name and add more details in the description If the auto commit is set to false, commit() is called for consistency adds unit tests Fix the check style issue of having more than 200 characters in single line

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.15
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit 8f50e90e6f5633376081a9fcfe260c4a6031ff85
Author: Vibhath Ileperuma <vi...@gmail.com>
AuthorDate: Sat Nov 27 20:43:51 2021 +0530

    NIFI-8605 Adding a new property for ExecuteSQL and ExecuteSQLRecord processors to enable/disable auto committing
    change the default value of auto commit function to true
    Changed the auto commit property name and add more details in the description
    If the auto commit is set to false, commit() is called for consistency
    adds unit tests
    Fix the check style issue of having more than 200 characters in single line
    
    Signed-off-by: Matthew Burgess <ma...@apache.org>
    
    This closes #5554
---
 .../processors/standard/AbstractExecuteSQL.java    | 339 +++++++++++----------
 .../nifi/processors/standard/ExecuteSQL.java       |   1 +
 .../nifi/processors/standard/ExecuteSQLRecord.java |   1 +
 .../nifi/processors/standard/TestExecuteSQL.java   |  17 ++
 .../processors/standard/TestExecuteSQLRecord.java  |  17 ++
 5 files changed, 217 insertions(+), 158 deletions(-)

diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java
index bd6962c..55a4326 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/AbstractExecuteSQL.java
@@ -169,6 +169,22 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor {
             .expressionLanguageSupported(ExpressionLanguageScope.FLOWFILE_ATTRIBUTES)
             .build();
 
+    public static final PropertyDescriptor AUTO_COMMIT = new PropertyDescriptor.Builder()
+            .name("esql-auto-commit")
+            .displayName("Set Auto Commit")
+            .description("Enables or disables the auto commit functionality of the DB connection. Default value is 'true'. " +
+                    "The default value can be used with most of the JDBC drivers and this functionality doesn't have any impact in most of the cases " +
+                    "since this processor is used to read data. " +
+                    "However, for some JDBC drivers such as PostgreSQL driver, it is required to disable the auto committing functionality " +
+                    "to limit the number of result rows fetching at a time. " +
+                    "When auto commit is enabled, postgreSQL driver loads whole result set to memory at once. " +
+                    "This could lead for a large amount of memory usage when executing queries which fetch large data sets. " +
+                    "More Details of this behaviour in PostgreSQL driver can be found in https://jdbc.postgresql.org//documentation/head/query.html. ")
+            .allowableValues("true", "false")
+            .defaultValue("true")
+            .required(true)
+            .build();
+
     protected List<PropertyDescriptor> propDescriptors;
 
     protected DBCPService dbcpService;
@@ -236,195 +252,202 @@ public abstract class AbstractExecuteSQL extends AbstractProcessor {
         }
 
         int resultCount = 0;
-        try (final Connection con = dbcpService.getConnection(fileToProcess == null ? Collections.emptyMap() : fileToProcess.getAttributes());
-             final PreparedStatement st = con.prepareStatement(selectQuery)) {
-            if (fetchSize != null && fetchSize > 0) {
-                try {
-                    st.setFetchSize(fetchSize);
-                } catch (SQLException se) {
-                    // Not all drivers support this, just log the error (at debug level) and move on
-                    logger.debug("Cannot set fetch size to {} due to {}", new Object[]{fetchSize, se.getLocalizedMessage()}, se);
+        try (final Connection con = dbcpService.getConnection(fileToProcess == null ? Collections.emptyMap() : fileToProcess.getAttributes())) {
+            con.setAutoCommit(context.getProperty(AUTO_COMMIT).asBoolean());
+            try (final PreparedStatement st = con.prepareStatement(selectQuery)) {
+                if (fetchSize != null && fetchSize > 0) {
+                    try {
+                        st.setFetchSize(fetchSize);
+                    } catch (SQLException se) {
+                        // Not all drivers support this, just log the error (at debug level) and move on
+                        logger.debug("Cannot set fetch size to {} due to {}", new Object[]{fetchSize, se.getLocalizedMessage()}, se);
+                    }
+                }
+                st.setQueryTimeout(queryTimeout); // timeout in seconds
+
+                // Execute pre-query, throw exception and cleanup Flow Files if fail
+                Pair<String, SQLException> failure = executeConfigStatements(con, preQueries);
+                if (failure != null) {
+                    // In case of failure, assigning config query to "selectQuery" to follow current error handling
+                    selectQuery = failure.getLeft();
+                    throw failure.getRight();
                 }
-            }
-            st.setQueryTimeout(queryTimeout); // timeout in seconds
-
-            // Execute pre-query, throw exception and cleanup Flow Files if fail
-            Pair<String,SQLException> failure = executeConfigStatements(con, preQueries);
-            if (failure != null) {
-                // In case of failure, assigning config query to "selectQuery" to follow current error handling
-                selectQuery = failure.getLeft();
-                throw failure.getRight();
-            }
 
-            if (fileToProcess != null) {
-                JdbcCommon.setParameters(st, fileToProcess.getAttributes());
-            }
-            logger.debug("Executing query {}", new Object[]{selectQuery});
+                if (fileToProcess != null) {
+                    JdbcCommon.setParameters(st, fileToProcess.getAttributes());
+                }
+                logger.debug("Executing query {}", new Object[]{selectQuery});
 
-            int fragmentIndex = 0;
-            final String fragmentId = UUID.randomUUID().toString();
+                int fragmentIndex = 0;
+                final String fragmentId = UUID.randomUUID().toString();
 
-            final StopWatch executionTime = new StopWatch(true);
+                final StopWatch executionTime = new StopWatch(true);
 
-            boolean hasResults = st.execute();
+                boolean hasResults = st.execute();
 
-            long executionTimeElapsed = executionTime.getElapsed(TimeUnit.MILLISECONDS);
+                long executionTimeElapsed = executionTime.getElapsed(TimeUnit.MILLISECONDS);
 
-            boolean hasUpdateCount = st.getUpdateCount() != -1;
+                boolean hasUpdateCount = st.getUpdateCount() != -1;
 
-            Map<String, String> inputFileAttrMap = fileToProcess == null ? null : fileToProcess.getAttributes();
-            String inputFileUUID = fileToProcess == null ? null : fileToProcess.getAttribute(CoreAttributes.UUID.key());
-            while (hasResults || hasUpdateCount) {
-                //getMoreResults() and execute() return false to indicate that the result of the statement is just a number and not a ResultSet
-                if (hasResults) {
-                    final AtomicLong nrOfRows = new AtomicLong(0L);
+                Map<String, String> inputFileAttrMap = fileToProcess == null ? null : fileToProcess.getAttributes();
+                String inputFileUUID = fileToProcess == null ? null : fileToProcess.getAttribute(CoreAttributes.UUID.key());
+                while (hasResults || hasUpdateCount) {
+                    //getMoreResults() and execute() return false to indicate that the result of the statement is just a number and not a ResultSet
+                    if (hasResults) {
+                        final AtomicLong nrOfRows = new AtomicLong(0L);
 
-                    try {
-                        final ResultSet resultSet = st.getResultSet();
-                        do {
-                            final StopWatch fetchTime = new StopWatch(true);
-
-                            FlowFile resultSetFF;
-                            if (fileToProcess == null) {
-                                resultSetFF = session.create();
-                            } else {
-                                resultSetFF = session.create(fileToProcess);
-                            }
+                        try {
+                            final ResultSet resultSet = st.getResultSet();
+                            do {
+                                final StopWatch fetchTime = new StopWatch(true);
 
-                            if (inputFileAttrMap != null) {
-                                resultSetFF = session.putAllAttributes(resultSetFF, inputFileAttrMap);
-                            }
+                                FlowFile resultSetFF;
+                                if (fileToProcess == null) {
+                                    resultSetFF = session.create();
+                                } else {
+                                    resultSetFF = session.create(fileToProcess);
+                                }
+
+                                if (inputFileAttrMap != null) {
+                                    resultSetFF = session.putAllAttributes(resultSetFF, inputFileAttrMap);
+                                }
 
 
-                            try {
-                                resultSetFF = session.write(resultSetFF, out -> {
-                                    try {
-                                        nrOfRows.set(sqlWriter.writeResultSet(resultSet, out, getLogger(), null));
-                                    } catch (Exception e) {
-                                        throw (e instanceof ProcessException) ? (ProcessException) e : new ProcessException(e);
+                                try {
+                                    resultSetFF = session.write(resultSetFF, out -> {
+                                        try {
+                                            nrOfRows.set(sqlWriter.writeResultSet(resultSet, out, getLogger(), null));
+                                        } catch (Exception e) {
+                                            throw (e instanceof ProcessException) ? (ProcessException) e : new ProcessException(e);
+                                        }
+                                    });
+
+                                    long fetchTimeElapsed = fetchTime.getElapsed(TimeUnit.MILLISECONDS);
+
+                                    // set attributes
+                                    final Map<String, String> attributesToAdd = new HashMap<>();
+                                    attributesToAdd.put(RESULT_ROW_COUNT, String.valueOf(nrOfRows.get()));
+                                    attributesToAdd.put(RESULT_QUERY_DURATION, String.valueOf(executionTimeElapsed + fetchTimeElapsed));
+                                    attributesToAdd.put(RESULT_QUERY_EXECUTION_TIME, String.valueOf(executionTimeElapsed));
+                                    attributesToAdd.put(RESULT_QUERY_FETCH_TIME, String.valueOf(fetchTimeElapsed));
+                                    attributesToAdd.put(RESULTSET_INDEX, String.valueOf(resultCount));
+                                    if (inputFileUUID != null) {
+                                        attributesToAdd.put(INPUT_FLOWFILE_UUID, inputFileUUID);
                                     }
-                                });
-
-                                long fetchTimeElapsed = fetchTime.getElapsed(TimeUnit.MILLISECONDS);
-
-                                // set attributes
-                                final Map<String, String> attributesToAdd = new HashMap<>();
-                                attributesToAdd.put(RESULT_ROW_COUNT, String.valueOf(nrOfRows.get()));
-                                attributesToAdd.put(RESULT_QUERY_DURATION, String.valueOf(executionTimeElapsed + fetchTimeElapsed));
-                                attributesToAdd.put(RESULT_QUERY_EXECUTION_TIME, String.valueOf(executionTimeElapsed));
-                                attributesToAdd.put(RESULT_QUERY_FETCH_TIME, String.valueOf(fetchTimeElapsed));
-                                attributesToAdd.put(RESULTSET_INDEX, String.valueOf(resultCount));
-                                if (inputFileUUID != null) {
-                                    attributesToAdd.put(INPUT_FLOWFILE_UUID, inputFileUUID);
-                                }
-                                attributesToAdd.putAll(sqlWriter.getAttributesToAdd());
-                                resultSetFF = session.putAllAttributes(resultSetFF, attributesToAdd);
-                                sqlWriter.updateCounters(session);
-
-                                // if fragmented ResultSet, determine if we should keep this fragment; set fragment attributes
-                                if (maxRowsPerFlowFile > 0) {
-                                    // if row count is zero and this is not the first fragment, drop it instead of committing it.
-                                    if (nrOfRows.get() == 0 && fragmentIndex > 0) {
-                                        session.remove(resultSetFF);
-                                        break;
+                                    attributesToAdd.putAll(sqlWriter.getAttributesToAdd());
+                                    resultSetFF = session.putAllAttributes(resultSetFF, attributesToAdd);
+                                    sqlWriter.updateCounters(session);
+
+                                    // if fragmented ResultSet, determine if we should keep this fragment; set fragment attributes
+                                    if (maxRowsPerFlowFile > 0) {
+                                        // if row count is zero and this is not the first fragment, drop it instead of committing it.
+                                        if (nrOfRows.get() == 0 && fragmentIndex > 0) {
+                                            session.remove(resultSetFF);
+                                            break;
+                                        }
+
+                                        resultSetFF = session.putAttribute(resultSetFF, FRAGMENT_ID, fragmentId);
+                                        resultSetFF = session.putAttribute(resultSetFF, FRAGMENT_INDEX, String.valueOf(fragmentIndex));
                                     }
 
-                                    resultSetFF = session.putAttribute(resultSetFF, FRAGMENT_ID, fragmentId);
-                                    resultSetFF = session.putAttribute(resultSetFF, FRAGMENT_INDEX, String.valueOf(fragmentIndex));
-                                }
-
-                                logger.info("{} contains {} records; transferring to 'success'", new Object[]{resultSetFF, nrOfRows.get()});
+                                    logger.info("{} contains {} records; transferring to 'success'", new Object[]{resultSetFF, nrOfRows.get()});
 
-                                // Report a FETCH event if there was an incoming flow file, or a RECEIVE event otherwise
-                                if(context.hasIncomingConnection()) {
-                                    session.getProvenanceReporter().fetch(resultSetFF, "Retrieved " + nrOfRows.get() + " rows", executionTimeElapsed + fetchTimeElapsed);
-                                } else {
-                                    session.getProvenanceReporter().receive(resultSetFF, "Retrieved " + nrOfRows.get() + " rows", executionTimeElapsed + fetchTimeElapsed);
-                                }
-                                resultSetFlowFiles.add(resultSetFF);
-
-                                // If we've reached the batch size, send out the flow files
-                                if (outputBatchSize > 0 && resultSetFlowFiles.size() >= outputBatchSize) {
-                                    session.transfer(resultSetFlowFiles, REL_SUCCESS);
-                                    // Need to remove the original input file if it exists
-                                    if (fileToProcess != null) {
-                                        session.remove(fileToProcess);
-                                        fileToProcess = null;
+                                    // Report a FETCH event if there was an incoming flow file, or a RECEIVE event otherwise
+                                    if (context.hasIncomingConnection()) {
+                                        session.getProvenanceReporter().fetch(resultSetFF, "Retrieved " + nrOfRows.get() + " rows", executionTimeElapsed + fetchTimeElapsed);
+                                    } else {
+                                        session.getProvenanceReporter().receive(resultSetFF, "Retrieved " + nrOfRows.get() + " rows", executionTimeElapsed + fetchTimeElapsed);
+                                    }
+                                    resultSetFlowFiles.add(resultSetFF);
+
+                                    // If we've reached the batch size, send out the flow files
+                                    if (outputBatchSize > 0 && resultSetFlowFiles.size() >= outputBatchSize) {
+                                        session.transfer(resultSetFlowFiles, REL_SUCCESS);
+                                        // Need to remove the original input file if it exists
+                                        if (fileToProcess != null) {
+                                            session.remove(fileToProcess);
+                                            fileToProcess = null;
+                                        }
+
+                                        session.commitAsync();
+                                        resultSetFlowFiles.clear();
                                     }
 
-                                    session.commitAsync();
-                                    resultSetFlowFiles.clear();
+                                    fragmentIndex++;
+                                } catch (Exception e) {
+                                    // Remove any result set flow file(s) and propagate the exception
+                                    session.remove(resultSetFF);
+                                    session.remove(resultSetFlowFiles);
+                                    if (e instanceof ProcessException) {
+                                        throw (ProcessException) e;
+                                    } else {
+                                        throw new ProcessException(e);
+                                    }
                                 }
+                            } while (maxRowsPerFlowFile > 0 && nrOfRows.get() == maxRowsPerFlowFile);
 
-                                fragmentIndex++;
-                            } catch (Exception e) {
-                                // Remove any result set flow file(s) and propagate the exception
-                                session.remove(resultSetFF);
-                                session.remove(resultSetFlowFiles);
-                                if (e instanceof ProcessException) {
-                                    throw (ProcessException) e;
-                                } else {
-                                    throw new ProcessException(e);
+                            // If we are splitting results but not outputting batches, set count on all FlowFiles
+                            if (outputBatchSize == 0 && maxRowsPerFlowFile > 0) {
+                                for (int i = 0; i < resultSetFlowFiles.size(); i++) {
+                                    resultSetFlowFiles.set(i,
+                                            session.putAttribute(resultSetFlowFiles.get(i), FRAGMENT_COUNT, Integer.toString(fragmentIndex)));
                                 }
                             }
-                        } while (maxRowsPerFlowFile > 0 && nrOfRows.get() == maxRowsPerFlowFile);
-
-                        // If we are splitting results but not outputting batches, set count on all FlowFiles
-                        if (outputBatchSize == 0 && maxRowsPerFlowFile > 0) {
-                            for (int i = 0; i < resultSetFlowFiles.size(); i++) {
-                                resultSetFlowFiles.set(i,
-                                        session.putAttribute(resultSetFlowFiles.get(i), FRAGMENT_COUNT, Integer.toString(fragmentIndex)));
-                            }
+                        } catch (final SQLException e) {
+                            throw new ProcessException(e);
                         }
-                    } catch (final SQLException e) {
-                        throw new ProcessException(e);
+
+                        resultCount++;
                     }
 
-                    resultCount++;
+                    // are there anymore result sets?
+                    try {
+                        hasResults = st.getMoreResults(Statement.CLOSE_CURRENT_RESULT);
+                        hasUpdateCount = st.getUpdateCount() != -1;
+                    } catch (SQLException ex) {
+                        hasResults = false;
+                        hasUpdateCount = false;
+                    }
                 }
 
-                // are there anymore result sets?
-                try {
-                    hasResults = st.getMoreResults(Statement.CLOSE_CURRENT_RESULT);
-                    hasUpdateCount = st.getUpdateCount() != -1;
-                } catch (SQLException ex) {
-                    hasResults = false;
-                    hasUpdateCount = false;
+                // Execute post-query, throw exception and cleanup Flow Files if fail
+                failure = executeConfigStatements(con, postQueries);
+                if (failure != null) {
+                    selectQuery = failure.getLeft();
+                    resultSetFlowFiles.forEach(ff -> session.remove(ff));
+                    throw failure.getRight();
                 }
-            }
 
-            // Execute post-query, throw exception and cleanup Flow Files if fail
-            failure = executeConfigStatements(con, postQueries);
-            if (failure != null) {
-                selectQuery = failure.getLeft();
-                resultSetFlowFiles.forEach(ff -> session.remove(ff));
-                throw failure.getRight();
-            }
-
-            // Transfer any remaining files to SUCCESS
-            session.transfer(resultSetFlowFiles, REL_SUCCESS);
-            resultSetFlowFiles.clear();
+                // If the auto commit is set to false, commit() is called for consistency
+                if (!con.getAutoCommit()) {
+                    con.commit();
+                }
 
-            //If we had at least one result then it's OK to drop the original file, but if we had no results then
-            //  pass the original flow file down the line to trigger downstream processors
-            if (fileToProcess != null) {
-                if (resultCount > 0) {
-                    session.remove(fileToProcess);
-                } else {
-                    fileToProcess = session.write(fileToProcess, out -> sqlWriter.writeEmptyResultSet(out, getLogger()));
-                    fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, "0");
-                    fileToProcess = session.putAttribute(fileToProcess, CoreAttributes.MIME_TYPE.key(), sqlWriter.getMimeType());
-                    session.transfer(fileToProcess, REL_SUCCESS);
+                // Transfer any remaining files to SUCCESS
+                session.transfer(resultSetFlowFiles, REL_SUCCESS);
+                resultSetFlowFiles.clear();
+
+                //If we had at least one result then it's OK to drop the original file, but if we had no results then
+                //  pass the original flow file down the line to trigger downstream processors
+                if (fileToProcess != null) {
+                    if (resultCount > 0) {
+                        session.remove(fileToProcess);
+                    } else {
+                        fileToProcess = session.write(fileToProcess, out -> sqlWriter.writeEmptyResultSet(out, getLogger()));
+                        fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, "0");
+                        fileToProcess = session.putAttribute(fileToProcess, CoreAttributes.MIME_TYPE.key(), sqlWriter.getMimeType());
+                        session.transfer(fileToProcess, REL_SUCCESS);
+                    }
+                } else if (resultCount == 0) {
+                    //If we had no inbound FlowFile, no exceptions, and the SQL generated no result sets (Insert/Update/Delete statements only)
+                    // Then generate an empty Output FlowFile
+                    FlowFile resultSetFF = session.create();
+
+                    resultSetFF = session.write(resultSetFF, out -> sqlWriter.writeEmptyResultSet(out, getLogger()));
+                    resultSetFF = session.putAttribute(resultSetFF, RESULT_ROW_COUNT, "0");
+                    resultSetFF = session.putAttribute(resultSetFF, CoreAttributes.MIME_TYPE.key(), sqlWriter.getMimeType());
+                    session.transfer(resultSetFF, REL_SUCCESS);
                 }
-            } else if (resultCount == 0) {
-                //If we had no inbound FlowFile, no exceptions, and the SQL generated no result sets (Insert/Update/Delete statements only)
-                // Then generate an empty Output FlowFile
-                FlowFile resultSetFF = session.create();
-
-                resultSetFF = session.write(resultSetFF, out -> sqlWriter.writeEmptyResultSet(out, getLogger()));
-                resultSetFF = session.putAttribute(resultSetFF, RESULT_ROW_COUNT, "0");
-                resultSetFF = session.putAttribute(resultSetFF, CoreAttributes.MIME_TYPE.key(), sqlWriter.getMimeType());
-                session.transfer(resultSetFF, REL_SUCCESS);
             }
         } catch (final ProcessException | SQLException e) {
             //If we had at least one result then it's OK to drop the original file, but if we had no results then
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
index b903e46..cc819db 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQL.java
@@ -134,6 +134,7 @@ public class ExecuteSQL extends AbstractExecuteSQL {
         pds.add(MAX_ROWS_PER_FLOW_FILE);
         pds.add(OUTPUT_BATCH_SIZE);
         pds.add(FETCH_SIZE);
+        pds.add(AUTO_COMMIT);
         propDescriptors = Collections.unmodifiableList(pds);
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQLRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQLRecord.java
index 76eeaae..2ebddac 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQLRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/ExecuteSQLRecord.java
@@ -140,6 +140,7 @@ public class ExecuteSQLRecord extends AbstractExecuteSQL {
         pds.add(MAX_ROWS_PER_FLOW_FILE);
         pds.add(OUTPUT_BATCH_SIZE);
         pds.add(FETCH_SIZE);
+        pds.add(AUTO_COMMIT);
         propDescriptors = Collections.unmodifiableList(pds);
     }
 
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java
index 16cd1d5..7227ce2 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQL.java
@@ -176,6 +176,18 @@ public class TestExecuteSQL {
     }
 
     @Test
+    public void testAutoCommitFalse() throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        runner.setProperty(ExecuteSQL.AUTO_COMMIT, "false");
+        invokeOnTrigger(null, QUERY_WITHOUT_EL, true, null, false);
+    }
+
+    @Test
+    public void testAutoCommitTrue() throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        runner.setProperty(ExecuteSQL.AUTO_COMMIT, "true");
+        invokeOnTrigger(null, QUERY_WITHOUT_EL, true, null, false);
+    }
+
+    @Test
     public void testWithNullIntColumn() throws SQLException {
         // remove previous test database, if any
         final File dbLocation = new File(DB_LOCATION);
@@ -556,6 +568,11 @@ public class TestExecuteSQL {
         SimpleCommerceDataSet.loadTestData2Database(con, 100, 200, 100);
         LOGGER.info("test data loaded");
 
+        //commit loaded data if auto-commit is dissabled
+        if (!con.getAutoCommit()){
+            con.commit();
+        }
+
         // ResultSet size will be 1x200x100 = 20 000 rows
         // because of where PER.ID = ${person.id}
         final int nrOfRows = 20000;
diff --git a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQLRecord.java b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQLRecord.java
index fe17e94..edf013f 100644
--- a/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQLRecord.java
+++ b/nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/test/java/org/apache/nifi/processors/standard/TestExecuteSQLRecord.java
@@ -170,6 +170,18 @@ public class TestExecuteSQLRecord {
     }
 
     @Test
+    public void testAutoCommitFalse() throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        runner.setProperty(ExecuteSQL.AUTO_COMMIT, "false");
+        invokeOnTriggerRecords(null, QUERY_WITHOUT_EL, true, null, false);
+    }
+
+    @Test
+    public void testAutoCommitTrue() throws InitializationException, ClassNotFoundException, SQLException, IOException {
+        runner.setProperty(ExecuteSQL.AUTO_COMMIT, "true");
+        invokeOnTriggerRecords(null, QUERY_WITHOUT_EL, true, null, false);
+    }
+
+    @Test
     public void testWithOutputBatching() throws InitializationException, SQLException {
         // remove previous test database, if any
         final File dbLocation = new File(DB_LOCATION);
@@ -545,6 +557,11 @@ public class TestExecuteSQLRecord {
         SimpleCommerceDataSet.loadTestData2Database(con, 100, 200, 100);
         LOGGER.info("test data loaded");
 
+        //commit loaded data if auto-commit is dissabled
+        if (!con.getAutoCommit()){
+            con.commit();
+        }
+
         // ResultSet size will be 1x200x100 = 20 000 rows
         // because of where PER.ID = ${person.id}
         final int nrOfRows = 20000;

[nifi] 03/04: NIFI-9524 exclude commons-logging and log4j-core banned dependencies for other build profiles as well (#5625)

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.15
in repository https://gitbox.apache.org/repos/asf/nifi.git

commit c71a18ff4770afe2d433d12b2582f54b98b02cb3
Author: Zoltán Kornél Török <zt...@cloudera.com>
AuthorDate: Tue Jan 4 15:14:04 2022 +0100

    NIFI-9524 exclude commons-logging and log4j-core banned dependencies for other build profiles as well (#5625)
---
 .../nifi-hadoop-libraries-nar/pom.xml              | 44 ++++++++++++++++------
 .../nifi-hive-bundle/nifi-hive3-nar/pom.xml        | 24 ++++++++++++
 .../nifi-ranger-bundle/nifi-ranger-plugin/pom.xml  | 16 ++++++++
 .../nifi-registry-ranger-plugin/pom.xml            | 16 ++++++++
 4 files changed, 88 insertions(+), 12 deletions(-)

diff --git a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
index 6521ef8..2eda8b1 100644
--- a/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hadoop-libraries-bundle/nifi-hadoop-libraries-nar/pom.xml
@@ -1,13 +1,13 @@
 <project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 https://maven.apache.org/xsd/maven-4.0.0.xsd">
-    <!-- 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 
+    <!-- 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. -->
     <modelVersion>4.0.0</modelVersion>
     <parent>
@@ -32,7 +32,7 @@
         <dependency>
             <groupId>org.apache.hadoop</groupId>
             <artifactId>hadoop-client</artifactId>
-            <version>${hadoop.version}</version>            
+            <version>${hadoop.version}</version>
             <exclusions>
                 <exclusion>
                     <groupId>com.google.code.findbugs</groupId>
@@ -59,12 +59,12 @@
         <dependency>
             <groupId>org.apache.avro</groupId>
             <artifactId>avro</artifactId>
-            <version>${avro.version}</version>        
+            <version>${avro.version}</version>
         </dependency>
         <dependency>
             <groupId>org.apache.avro</groupId>
             <artifactId>avro-ipc</artifactId>
-            <version>${avro.version}</version>        
+            <version>${avro.version}</version>
         </dependency>
         <dependency>
             <groupId>com.github.stephenc.findbugs</groupId>
@@ -108,6 +108,10 @@
                             <groupId>com.fasterxml.jackson.core</groupId>
                             <artifactId>jackson-core</artifactId>
                         </exclusion>
+                        <exclusion>
+                            <groupId>commons-logging</groupId>
+                            <artifactId>commons-logging</artifactId>
+                        </exclusion>
                     </exclusions>
                 </dependency>
                 <dependency>
@@ -134,6 +138,12 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-cloud-storage</artifactId>
                     <version>${hadoop.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
             </dependencies>
         </profile>
@@ -148,6 +158,16 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-ozone-client</artifactId>
                     <version>${ozone.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                      <exclusion>
+                          <groupId>org.apache.logging.log4j</groupId>
+                          <artifactId>log4j-core</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>
diff --git a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
index 6a78c0c..81e4327 100644
--- a/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
+++ b/nifi-nar-bundles/nifi-hive-bundle/nifi-hive3-nar/pom.xml
@@ -67,6 +67,10 @@
             </activation>
             <dependencies>
                 <dependency>
+                    <groupId>org.slf4j</groupId>
+                    <artifactId>jcl-over-slf4j</artifactId>
+                </dependency>
+                <dependency>
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-azure</artifactId>
                     <version>${hadoop.version}</version>
@@ -79,6 +83,10 @@
                             <groupId>com.fasterxml.jackson.core</groupId>
                             <artifactId>jackson-core</artifactId>
                         </exclusion>
+                        <exclusion>
+                            <groupId>commons-logging</groupId>
+                            <artifactId>commons-logging</artifactId>
+                        </exclusion>
                     </exclusions>
                 </dependency>
                 <dependency>
@@ -105,6 +113,12 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-cloud-storage</artifactId>
                     <version>${hadoop.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
             </dependencies>
         </profile>
@@ -119,6 +133,16 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-ozone-client</artifactId>
                     <version>${ozone.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                      <exclusion>
+                          <groupId>org.apache.logging.log4j</groupId>
+                          <artifactId>log4j-core</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>
diff --git a/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/pom.xml b/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/pom.xml
index f07a5ca..7c82e85 100644
--- a/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/pom.xml
+++ b/nifi-nar-bundles/nifi-ranger-bundle/nifi-ranger-plugin/pom.xml
@@ -264,6 +264,12 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-cloud-storage</artifactId>
                     <version>${ranger.hadoop.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
             </dependencies>
         </profile>
@@ -278,6 +284,16 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-ozone-client</artifactId>
                     <version>${ozone.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                      <exclusion>
+                          <groupId>org.apache.logging.log4j</groupId>
+                          <artifactId>log4j-core</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>
diff --git a/nifi-registry/nifi-registry-extensions/nifi-registry-ranger/nifi-registry-ranger-plugin/pom.xml b/nifi-registry/nifi-registry-extensions/nifi-registry-ranger/nifi-registry-ranger-plugin/pom.xml
index 16994c4..956d512 100644
--- a/nifi-registry/nifi-registry-extensions/nifi-registry-ranger/nifi-registry-ranger-plugin/pom.xml
+++ b/nifi-registry/nifi-registry-extensions/nifi-registry-ranger/nifi-registry-ranger-plugin/pom.xml
@@ -335,6 +335,12 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-cloud-storage</artifactId>
                     <version>${ranger.hadoop.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
             </dependencies>
         </profile>
@@ -349,6 +355,16 @@
                     <groupId>org.apache.hadoop</groupId>
                     <artifactId>hadoop-ozone-client</artifactId>
                     <version>${ranger.ozone.version}</version>
+                    <exclusions>
+                      <exclusion>
+                          <groupId>commons-logging</groupId>
+                          <artifactId>commons-logging</artifactId>
+                      </exclusion>
+                      <exclusion>
+                          <groupId>org.apache.logging.log4j</groupId>
+                          <artifactId>log4j-core</artifactId>
+                      </exclusion>
+                    </exclusions>
                 </dependency>
                 <dependency>
                     <groupId>org.apache.hadoop</groupId>