You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@nifi.apache.org by GitBox <gi...@apache.org> on 2022/01/20 18:39:20 UTC

[GitHub] [nifi] tpalfy opened a new pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

tpalfy opened a new pull request #5692:
URL: https://github.com/apache/nifi/pull/5692


   <!--
     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.
   -->
   
   https://issues.apache.org/jira/browse/NIFI-9609
   
   ### For all changes:
   - [ ] Is there a JIRA ticket associated with this PR? Is it referenced 
        in the commit message?
   
   - [ ] Does your PR title start with **NIFI-XXXX** where XXXX is the JIRA number you are trying to resolve? Pay particular attention to the hyphen "-" character.
   
   - [ ] Has your PR been rebased against the latest commit within the target branch (typically `main`)?
   
   - [ ] Is your initial contribution a single, squashed commit? _Additional commits in response to PR reviewer feedback should be made on this branch and pushed to allow change tracking. Do not `squash` or use `--force` when pushing to allow for clean monitoring of changes._
   
   ### For code changes:
   - [ ] Have you ensured that the full suite of tests is executed via `mvn -Pcontrib-check clean install` at the root `nifi` folder?
   - [ ] Have you written or updated unit tests to verify your changes?
   - [ ] Have you verified that the full build is successful on JDK 8?
   - [ ] Have you verified that the full build is successful on JDK 11?
   - [ ] If adding new dependencies to the code, are these dependencies licensed in a way that is compatible for inclusion under [ASF 2.0](http://www.apache.org/legal/resolved.html#category-a)?
   - [ ] If applicable, have you updated the `LICENSE` file, including the main `LICENSE` file under `nifi-assembly`?
   - [ ] If applicable, have you updated the `NOTICE` file, including the main `NOTICE` file found under `nifi-assembly`?
   - [ ] If adding new Properties, have you added `.displayName` in addition to .name (programmatic access) for each of the new properties?
   
   ### For documentation related changes:
   - [ ] Have you ensured that format looks appropriate for the output in which it is rendered?
   
   ### Note:
   Please ensure that once the PR is submitted, you check GitHub Actions CI for build issues and submit an update to your PR as soon as possible.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1017815068


   This PR appears to introduce a large amount of code duplication from the standard DBCPConnectionPool service just to register the Snowflake Driver.  Does standard DBCPConnectionPool service not work with the Snowflake Computing JDBC driver?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy removed a comment on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy removed a comment on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1032838536


   LGTM merged to main


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip edited a comment on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip edited a comment on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1039276217


   @tpalfy Please do not forget to add LICENSE / NOTICE files in the nar bundle.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1047209429


   @exceptionfactory It looks good to me now. I also tested it a bit with Snowflake and it works properly.
   There are two minor typos in the description of `Snowflake URL` property: _"can can have"_ and _"TABLe"_
   These could also be fixed during the merge.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1019942580


   @joewitt @exceptionfactory 
   I was thinking about  avoiding code duplication using inheritance.
   
   However that would require a dependency on the _nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service_ **jar** or the _nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service-nar_ **nar**.
   
   I had the (maybe unfounded) feeling that such a dependency would be undesirable. When I looked how other similar modules were created I found no example of a similar dependency, there were code duplications instead. That's why I assumed there is a reason why we don't want such dependencies in general.
   
   Is it okay to add either the **jar** or the **nar** dependency?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1022273920


   The `getDriver()` method appears to be exactly the same as the `DBCPConnectionPool`, so perhaps I am missing how duplicating it changes the runtime behavior.
   
   Although there may be some marginal utility in having an explicit controller service, I am not in favor of the current approach without more significant code reduction of duplication.  If that requires some refactoring of `DBCPConnectionPool` to enable clean extension, that seems better than duplicating property descriptors and configuration methods.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1021512187


   I thought it's a good idea to separate ```.name()```  and ```.displayName()```. With that I'd leave the ```onConfigured()``` as well. There's no real business logic duplication just straightforward configuration. I think that is fine.
   I can remove the ```getSupportedDynamicPropertyDescriptor``` but I left the ```getDriver``` method intentionally because the ```Class.forName``` needs to be called there, otherwise it's not be able to find the driver.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792876669



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {
+        final Class<?> clazz;
+
+        try {
+            clazz = Class.forName(driverName);
+        } catch (final ClassNotFoundException e) {
+            throw new ProcessException("Driver class " + driverName +  " is not found", e);
+        }
+
+        try {
+            return DriverManager.getDriver(url);
+        } catch (final SQLException e) {
+            // In case the driver is not registered by the implementation, we explicitly try to register it.
+            try {
+                final Driver driver = (Driver) clazz.newInstance();
+                DriverManager.registerDriver(driver);
+                return DriverManager.getDriver(url);
+            } catch (final SQLException e2) {
+                throw new ProcessException("No suitable driver for the given Database Connection URL", e2);
+            } catch (final IllegalAccessException | InstantiationException e2) {
+                throw new ProcessException("Creating driver instance is failed", e2);
+            }
+        }
+    }
+
+    /**
+     * Shutdown pool, close all open connections.
+     *
+     * @throws SQLException if there is an error while closing open connections
+     */
+    @OnDisabled
+    public void shutdown() throws SQLException {
+        try {
+            if (dataSource != null) {
+                dataSource.close();
+            }
+        } finally {
+            dataSource = null;
+        }
+    }
+
+    @Override
+    public Connection getConnection() throws ProcessException {
+        try {
+            final Connection connection = dataSource.getConnection();
+            return connection;
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return this.getClass().getSimpleName() + "[id=" + getIdentifier() + "]";
+    }

Review comment:
       Sounds good.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792874712



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;

Review comment:
       Yeah, making it protected seems to be a good idea.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792873690



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/pom.xml
##########
@@ -0,0 +1,83 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-snowflake-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-snowflake-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-dbcp-service-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-dbcp-service</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-credentials-service-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-user-service-api</artifactId>
+        </dependency>

Review comment:
       The problem is that static constants in ```DBCPConnectionPool``` reference kerberos-related classes. So without these we can't load the new service class.
   Of course this wouldn't be necessary if we didn't extend the ```DBCPConnectionPool```.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792868930



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")

Review comment:
       I think this annotation should stay and ```IT``` postfix should be preserved for integration tests that can be run at any time with the appropriate maven runtime settings without manual intervention.
   
   We already use this pattern and I think it is a good one.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020456025






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1032838536


   LGTM merged to main


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1033105888


   One other point of consideration, the `snowflake-jdbc` JAR is almost 28 MB due to shading a large number of dependencies. In light of current sizing limitations on the standard NiFi binary, it seems like this should not be part of the standard assembly.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020467938


   I understand what this approach would achieve and how.
   What I'm trying to say is that the ```nifi-dbcp-shared``` would hold a full-fledged service all the rest. It wouldn't look like a utility module.
   
   And about the general issue - we have code duplications instead of this approach already. Why is this so case so different?
   
   That being said, maybe we could do something that is technically the same but the concept is a bit different.
   We split the ```nifi-dbcp-service``` into to modules. One can remain ```nifi-dbcp-service``` and holds the code and we can have a ```nifi-dbcp-service-meta``` module that can have the META-INF.
   
   I understand that in this case the result would basically be the same but with this we can create a pattern that can be used generally for any future or existing module allow us to reuse any implementation code.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1022303168


   Regarding the ```getDriver()``` can you please help me understand which part needs further explanation in this:
   
   > The reason why we can't rely on the DBCPConnectionPool.getDriver() is that the subsequent Class.forName() call uses the classloader of the caller class. Which would be the nifi-dbcp-service NAR classloader which doesn't see the snowflake driver as that is included in the nifi-snowflake-services NAR. 
   
   I'm really not sure how I could explain better.
   
   As for the concerns about code duplication I think "more significant code reduction of duplication" lacks sufficient specificity. I reduced code duplication already significantly and addressed the remaining ones in great detail in my previous comments.
   With that I'll consider this argument void for vagueness.
   
   I feel like we are just running around in circles with little to no benefit. I'll leave the implementation the way as it is now.
   
   I'll consider subsequent suggestions if they are sufficiently specific and the reviewer can confirm their validity by trying them out in a running environment. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r793821940



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       I'm not sure it's a good idea to bypass the ```DriverManager``` though.
   I think we should keep the logic that returns the (same) registered driver for each call (and also checks that the URL is compatible).
   
   That's really all the ```DBCPConnectionPool.getDriver()``` itself does with some exception handling.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020451641






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip edited a comment on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip edited a comment on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1039276217


   @tpalfy Please do not forget to add LICENSE / NOTICE files in the nar bundle.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] asfgit closed pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #5692:
URL: https://github.com/apache/nifi/pull/5692


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r793802743



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       I was able to configure and enable the `SnowflakeComputingConnectionPool` service and attempt a query using `ExecuteSQL` without any incompatible errors.
   
   As an even better solution, instead of adding `cloneAncestorResources = true`, overriding the `getDriver()` method to return a new instance of the `SnowflakeDriver` class also appears to work, since it avoids the `Class.forName()` lookup in `DBCPConnectionPool`.  The method in the SnowflakeComputing class would read as follows:
   
   ```suggestion
       @Override
       protected Driver getDriver(final String driverName, final String url) {
           return new SnowflakeDriver();
       }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792875797



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {
+        final Class<?> clazz;
+
+        try {
+            clazz = Class.forName(driverName);
+        } catch (final ClassNotFoundException e) {
+            throw new ProcessException("Driver class " + driverName +  " is not found", e);
+        }
+
+        try {
+            return DriverManager.getDriver(url);
+        } catch (final SQLException e) {
+            // In case the driver is not registered by the implementation, we explicitly try to register it.
+            try {
+                final Driver driver = (Driver) clazz.newInstance();
+                DriverManager.registerDriver(driver);
+                return DriverManager.getDriver(url);
+            } catch (final SQLException e2) {
+                throw new ProcessException("No suitable driver for the given Database Connection URL", e2);
+            } catch (final IllegalAccessException | InstantiationException e2) {
+                throw new ProcessException("Creating driver instance is failed", e2);
+            }
+        }
+    }
+
+    /**
+     * Shutdown pool, close all open connections.
+     *
+     * @throws SQLException if there is an error while closing open connections
+     */
+    @OnDisabled
+    public void shutdown() throws SQLException {
+        try {
+            if (dataSource != null) {
+                dataSource.close();
+            }
+        } finally {
+            dataSource = null;
+        }
+    }
+
+    @Override
+    public Connection getConnection() throws ProcessException {
+        try {
+            final Connection connection = dataSource.getConnection();
+            return connection;
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
+        }
+    }

Review comment:
       Yeah, sounds reasonable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r801650101



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolIT.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_USER<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+public class SnowflakeConnectionPoolIT {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";

Review comment:
       The specific URL should be changed to something generic, perhaps `hostname.snowflakecomputing.com`.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }

Review comment:
       Following up on a previous comment, recommend removing the re-declaration of these property descriptors and changing the approach to reuse the descriptors from the parent class. Instead of declaring the same descriptors, it should be possible to call `super.getSupportedPropertyDecriptors()` to create a new array, then remove the `Database Driver Class Name`, `Database Driver Locations`, and Kerberos properties. The other approach would be to reuse the same descriptor variables and just add the necessary items.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);

Review comment:
       The majority of this method is duplicative of the parent method and should be refactored. The Kerberos-specific elements could be refactored in the superclass to allow reuse.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r801752690



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }

Review comment:
       As stated before I disagree with this.
   2 parts of this:
   1. The properties themselves. I created new ones so that we can distinguish between `name` and `displayeName`.
   2. Even if I used the same properties, a restructuring driven by the snowflake-specific use-case is not desirable conceptually and actually would result in a much more messy code in practice.
   
   I'm willing to reconsider if I saw a specific suggestion and the rest of the community (at least one more contributor) would vote for that implementation.
   

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,265 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);

Review comment:
       As stated before if I keep the new propertyDescriptors (to be able to have separate name and displayName for them) then the shared code would require a refactor that would introduce the same amount of complexity we try to prevent with the refactor in the first place.
   I could create a method something like setupDatasource but it would get a bunch of parameters or would get a DTO which I would need to call getters and setters on - just like on the datasource itself.
   
   Again, it's not business logic but a sequence of getter and setter calls. No need to overengineer it.
   
   I'm willing to reconsider if I saw a specific suggestion and the rest of the community (at least one more contributor) would vote for that implementation.
   
   

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolIT.java
##########
@@ -0,0 +1,86 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_USER<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+public class SnowflakeConnectionPoolIT {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";

Review comment:
       The far most important here is to help the reader understand the intent.
   While the `hostname.snowflakecomputing.com` would make sense if we wanted to make this abstract but that's not the case.
   No point in making this abstract. We want to make this understandable. And in this case a real-life is example that would be immediately familiar to a user is best suited for that purpose.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1017832265


   I think part of the rub here David is that the existing stuff does work.  Requires a bit more user steps/configuration and then folks just assume it is different and don't think it works well with Snowflake.  But it does.  So this helps ease the user configuration burden and puts up a big flashing sign "hey yep nifi works great for sending data to snowflake".  Two birds...one PR.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020456025


   The `nifi-dbcp-shared` would contain the shared service implementation code, but not the service definition file, so that would be the difference from the current structure.
   
   This is somewhat of an odd case since it is essentially providing a convenience class for something that can be configured using the existing controller service.  Although there may not be good examples of this in the current code base, this is an opportunity to avoid unnecessary code duplication.
   
   There may be other module structure alternatives, but one way or the other, we should avoid this level of code duplication.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792819329



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")

Review comment:
       This annotation should be removed and the class name should be changed to end with `IT` so it will be treated as an integration test and ignored for standard unit test execution.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;

Review comment:
       Changing the `DBCPConnectionPool` member `dataSource` to `protected`, or adding an accessor method would avoid having to declare this again.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")
+public class SnowflakeConnectionPoolTest {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";
+//    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com/?db=test_db";
+//    public static final String SNOWFLAKE_URL = "jdbc:snowflake://tm55946.us-east-2.aws.snowflakecomputing.com";

Review comment:
       Recommend replacing commented lines and specific URL with some other placeholder value.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/pom.xml
##########
@@ -0,0 +1,83 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<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 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <modelVersion>4.0.0</modelVersion>
+
+    <parent>
+        <groupId>org.apache.nifi</groupId>
+        <artifactId>nifi-snowflake-bundle</artifactId>
+        <version>1.16.0-SNAPSHOT</version>
+    </parent>
+
+    <artifactId>nifi-snowflake-services</artifactId>
+    <packaging>jar</packaging>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-dbcp-service-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-dbcp-service</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-credentials-service-api</artifactId>
+            <version>1.16.0-SNAPSHOT</version>
+            <scope>provided</scope>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.nifi</groupId>
+            <artifactId>nifi-kerberos-user-service-api</artifactId>
+        </dependency>

Review comment:
       Are these dependencies necessary? It looks like they could be removed since there are no direct Kerberos references.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {
+        final Class<?> clazz;
+
+        try {
+            clazz = Class.forName(driverName);
+        } catch (final ClassNotFoundException e) {
+            throw new ProcessException("Driver class " + driverName +  " is not found", e);
+        }
+
+        try {
+            return DriverManager.getDriver(url);
+        } catch (final SQLException e) {
+            // In case the driver is not registered by the implementation, we explicitly try to register it.
+            try {
+                final Driver driver = (Driver) clazz.newInstance();
+                DriverManager.registerDriver(driver);
+                return DriverManager.getDriver(url);
+            } catch (final SQLException e2) {
+                throw new ProcessException("No suitable driver for the given Database Connection URL", e2);
+            } catch (final IllegalAccessException | InstantiationException e2) {
+                throw new ProcessException("Creating driver instance is failed", e2);
+            }
+        }
+    }
+
+    /**
+     * Shutdown pool, close all open connections.
+     *
+     * @throws SQLException if there is an error while closing open connections
+     */
+    @OnDisabled
+    public void shutdown() throws SQLException {
+        try {
+            if (dataSource != null) {
+                dataSource.close();
+            }
+        } finally {
+            dataSource = null;
+        }
+    }
+
+    @Override
+    public Connection getConnection() throws ProcessException {
+        try {
+            final Connection connection = dataSource.getConnection();
+            return connection;
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
+        }
+    }
+
+    @Override
+    public String toString() {
+        return this.getClass().getSimpleName() + "[id=" + getIdentifier() + "]";
+    }

Review comment:
       Changing `DBCPConnectionPool.toString()` to use the `getClass.getSimpleName()` approach would avoid having to override the method in this class.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")
+public class SnowflakeConnectionPoolTest {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";
+//    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com/?db=test_db";
+//    public static final String SNOWFLAKE_URL = "jdbc:snowflake://tm55946.us-east-2.aws.snowflakecomputing.com";
+    public static final String SNOWFLAKE_USER = "???";
+    public static final String SNOWFLAKE_PASSWORD = "???";
+    public static final String TABLE_NAME = "test_db.public.test_table";
+
+    private static final String SERVICE_ID = SnowflakeComputingConnectionPool.class.getName();
+
+    private TestRunner runner;
+
+    private SnowflakeComputingConnectionPool service;
+
+    @Test
+    void testReadSnowflakeTable() throws Exception {
+        runner.assertValid(service);
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+
+        try (
+            final Connection connection = service.getConnection();
+            final Statement st = connection.createStatement()
+        ) {
+            ResultSet resultSet = st.executeQuery("select * from " + TABLE_NAME);
+
+            int nrOfRows = 0;
+            while (resultSet.next()) {
+                nrOfRows++;
+            }
+
+            System.out.println("Read " + nrOfRows + " records from " + TABLE_NAME);

Review comment:
       Recommend removing `System.out.println()` and replacing with an assertion indicating some number of expected rows.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {
+        final Class<?> clazz;
+
+        try {
+            clazz = Class.forName(driverName);
+        } catch (final ClassNotFoundException e) {
+            throw new ProcessException("Driver class " + driverName +  " is not found", e);
+        }
+
+        try {
+            return DriverManager.getDriver(url);
+        } catch (final SQLException e) {
+            // In case the driver is not registered by the implementation, we explicitly try to register it.
+            try {
+                final Driver driver = (Driver) clazz.newInstance();
+                DriverManager.registerDriver(driver);
+                return DriverManager.getDriver(url);
+            } catch (final SQLException e2) {
+                throw new ProcessException("No suitable driver for the given Database Connection URL", e2);
+            } catch (final IllegalAccessException | InstantiationException e2) {
+                throw new ProcessException("Creating driver instance is failed", e2);
+            }
+        }
+    }
+
+    /**
+     * Shutdown pool, close all open connections.
+     *
+     * @throws SQLException if there is an error while closing open connections
+     */
+    @OnDisabled
+    public void shutdown() throws SQLException {
+        try {
+            if (dataSource != null) {
+                dataSource.close();
+            }
+        } finally {
+            dataSource = null;
+        }
+    }
+
+    @Override
+    public Connection getConnection() throws ProcessException {
+        try {
+            final Connection connection = dataSource.getConnection();
+            return connection;
+        } catch (final SQLException e) {
+            throw new ProcessException(e);
+        }
+    }

Review comment:
       The `shutdown` and `getConnection` methods in `DBCPConnectionPool` should work since they perform conditional checks on the Kerberos user property and otherwise implement the same behavior.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       Overriding this method to call `super.getDriver()` with `SnowflakeDriver.class.getName()` as the first argument should avoid having to duplicate the code.  In runtime testing, adding `cloneAncestorResources = true` to the `RequiresInstanceClassLoading` annotation resolved the issue with driver class not found exceptions.

##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }

Review comment:
       Although this is convenient, it seems better to require one form or the other to avoid potential confusion.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
joewitt commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792872601



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")
+public class SnowflakeConnectionPoolTest {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";
+//    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com/?db=test_db";
+//    public static final String SNOWFLAKE_URL = "jdbc:snowflake://tm55946.us-east-2.aws.snowflakecomputing.com";

Review comment:
       Let's make sure these examples show up in the documentation/additional documentation for the processor.  Let's not use the code for that.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1021056412


   @tpalfy @exceptionfactory Regarding nar/jar dependencies, I believe the following should work:
   - declare `nifi-dbcp-service-nar` as the parent nar of the new `nifi-snowflake-service-nar` bundle (all classes in jars bundled in the parent will be available for the child at runtime)
   - add `nifi-dbcp-service` (jar) dependency in `nifi-snowflake-service` as `provided` (=> you can inherit from the class but ServiceLoader will not initialize duplicated service classes)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1021522662


   Is there a reason this class is unable to use the `getDriver()` method from `DBCPConnectionPool`? The methods appear to be the same.  Although there may not be business logic duplication in `onConfigured()`, there is still a lot of setup code that should be shared.  The same applies to the property descriptor definitions.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792880955



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")

Review comment:
       There are a handful of disabled tests that should be renamed to integration tests, so we should not continue that pattern.  Disabled tests show up as warnings during builds, which is another reason to keep a clearer distinction between automated unit tests and manual integration tests.  If this test could be run in an automated fashion, that would be great, but it does not work that way as it stands.
   
   As an alternative solution, JUnit 5 supports conditional annotations that would enable the test when environment variables or system properties are present.  Adjusting to test to read the URL and credentials from environment variables would allow keeping this as a unit test, but making it conditional on the presence of those variables.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r793742152



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       For me this change results in an "Incompatible Controller Service Configured" error on the processor.
   Were you able to run a successful query with this setup? Is there something else that needs to be changed?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1032869488


   Let me try to explain in more detail my concerns through a demonstration.
   
   Here's how I would imagine the propertyDescriptor-related duplication removal:
   1. We would need to settle for the properties not to have different `name` and `displayName`. Btw this is a decision that is basically final. We won't be able back down on that.
   2. Here's how the code would look like:
   Instead of this:
   ```java
       private static final List<PropertyDescriptor> properties;
   
       static {
           final List<PropertyDescriptor> props = new ArrayList<>();
           props.add(SNOWFLAKE_URL);
           props.add(SNOWFLAKE_USER);
           props.add(SNOWFLAKE_PASSWORD);
           props.add(VALIDATION_QUERY);
           props.add(MAX_WAIT_TIME);
           props.add(MAX_TOTAL_CONNECTIONS);
           props.add(MIN_IDLE);
           props.add(MAX_IDLE);
           props.add(MAX_CONN_LIFETIME);
           props.add(EVICTION_RUN_PERIOD);
           props.add(MIN_EVICTABLE_IDLE_TIME);
           props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
   
           properties = Collections.unmodifiableList(props);
       }
   
       @Override
       protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
           return properties;
       }
   ```
   We would have this:
   ```java
       @Override
       protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
           final List<PropertyDescriptor> props = new ArrayList<>();
   
           Collection<PropertyDescriptor> becauseImSnowflakeIDontNeedTheseProperties = Arrays.asList(
               DATABASE_URL,
               DB_DRIVERNAME,
               DB_DRIVER_LOCATION,
               KERBEROS_USER_SERVICE,
               KERBEROS_CREDENTIALS_SERVICE,
               KERBEROS_PRINCIPAL,
               KERBEROS_PASSWORD,
               DB_USER,
               DB_PASSWORD,
               VALIDATION_QUERY
           );
   
           props.add(SNOWFLAKE_URL);
           props.add(SNOWFLAKE_USER);
           props.add(SNOWFLAKE_PASSWORD);
           props.add(VALIDATION_QUERY);
           props.addAll(super.getSupportedPropertyDescriptors());
           props.removeAll(becauseImSnowflakeIDontNeedTheseProperties);
   
           return props;
       }
   ```
   To me there's no question that the former is better even if we agree to 1.
   
   With the datasource duplication we would do something like this:
   In  DBCPConnectionPool we would add the following method: 
   ```java
       protected void setDatasourceSettingsExceptDriverOfCourse(
           String dburl,
           String user,
           String passw,
           Integer maxTotal,
           Long maxWaitMillis,
           Integer minIdle,
           Integer maxIdle,
           Long maxConnLifetimeMillis,
           Long timeBetweenEvictionRunsMillis,
           Long minEvictableIdleTimeMillis,
           Long softMinEvictableIdleTimeMillis
       ) {
           dataSource.setUrl(dburl);
           dataSource.setUsername(user);
           dataSource.setPassword(passw);
           dataSource.setMaxWaitMillis(maxWaitMillis);
           dataSource.setMaxTotal(maxTotal);
           dataSource.setMinIdle(minIdle);
           dataSource.setMaxIdle(maxIdle);
           dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
           dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
           dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
           dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
       }
   ```
   And instead of this:
   ```java
           dataSource = new BasicDataSource();
           dataSource.setDriver(getDriver(driverName, dburl));
           dataSource.setMaxWaitMillis(maxWaitMillis);
           dataSource.setMaxTotal(maxTotal);
           dataSource.setMinIdle(minIdle);
           dataSource.setMaxIdle(maxIdle);
           dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
           dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
           dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
           dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
   
           if (validationQuery!=null && !validationQuery.isEmpty()) {
               dataSource.setValidationQuery(validationQuery);
               dataSource.setTestOnBorrow(true);
           }
   
           dataSource.setUrl(dburl);
           dataSource.setUsername(user);
           dataSource.setPassword(passw);
   ```
   We would have this:
   ```java
           dataSource = new BasicDataSource();
           dataSource.setDriver(getDriver(driverName, dburl));
           setDatasourceSettingsExceptDriverOfCourse(
               dburl,
               user,
               passw,
               maxTotal,
               maxWaitMillis,
               minIdle,
               maxIdle,
               maxConnLifetimeMillis,
               timeBetweenEvictionRunsMillis,
               minEvictableIdleTimeMillis,
               softMinEvictableIdleTimeMillis
           );
   ```
   And in SnowflakeComputingConnectionPool instead of this:
   ```java
           dataSource.setUrl(connectionString);
           dataSource.setUsername(user);
           dataSource.setPassword(password);
   
           dataSource.setMaxWaitMillis(maxWaitMillis);
           dataSource.setMaxTotal(maxTotal);
           dataSource.setMinIdle(minIdle);
           dataSource.setMaxIdle(maxIdle);
           dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
           dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
           dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
           dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
   ```
   We would have this:
   ```java
           setDatasourceSettingsExceptDriverOfCourse(
               connectionString,
               user,
               password,
               maxTotal,
               maxWaitMillis,
               minIdle,
               maxIdle,
               maxConnLifetimeMillis,
               timeBetweenEvictionRunsMillis,
               minEvictableIdleTimeMillis,
               softMinEvictableIdleTimeMillis
           );
   ```
   To me this doesn't feel an improvement.
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r804880705



##########
File path: nifi-nar-bundles/nifi-standard-services/nifi-dbcp-service-bundle/nifi-dbcp-service/src/main/java/org/apache/nifi/dbcp/DBCPConnectionPool.java
##########
@@ -81,30 +81,30 @@
     protected static final String SENSITIVE_PROPERTY_PREFIX = "SENSITIVE.";
 
     /**
-     * Copied from {@link GenericObjectPoolConfig.DEFAULT_MIN_IDLE} in Commons-DBCP 2.7.0
+     * Copied from {@link GenericObjectPoolConfig#DEFAULT_MIN_IDLE} in Commons-DBCP 2.7.0
      */
-    private static final String DEFAULT_MIN_IDLE = "0";
+    protected static final String DEFAULT_MIN_IDLE = "0";

Review comment:
       It is unused now because `GenericObjectPoolConfig.DEFAULT_MIN_IDLE` is directly referenced in the `PropertyDescriptor`.
   Not clear why these constants were copied from `GenericObjectPoolConfig` but `DEFAULT_MAX_IDLE` could also be used directly. Maybe others too.
   I'm not in favour of changing them, I'm rather unsure if DEFAULT_MIN_IDLE should be changed at all.
   
   The constants are not referenced any more in the subclass directly, so they could remain `private` I think.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1039276217


   @tpalfy Please do not forget to add LICENCE / NOTICE files in the nar bundle.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1039276217


   @tpalfy Please do not forget to add LICENCE / NOTICE files in the nar bundle.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] turcsanyip commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
turcsanyip commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1021056412


   @tpalfy @exceptionfactory Regarding nar/jar dependencies, I believe the following should work:
   - declare `nifi-dbcp-service-nar` as the parent nar of the new `nifi-snowflake-service-nar` bundle (all classes in jars bundled in the parent will be available for the child at runtime)
   - add `nifi-dbcp-service` (jar) dependency in `nifi-snowflake-service` as `provided` (=> you can inherit from the class but ServiceLoader will not initialize duplicated service classes)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792877954



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       Wouldn't this make the new module expose the ```DBCPConnectionPool``` service a second time?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792881987



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }

Review comment:
       I think both ways have a fairly strong justification.
   The logic and its purpose behind it is sufficiently simple in my opinion to consider it easily understandable.
   I'm more concerned about the extra complexity but that is small too and I think it's worth it all things considered.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1022104731


   The reason why we can't rely on the ```DBCPConnectionPool.getDriver()``` is that the subsequent ```Class.forName()``` call uses the classloader of the caller class. Which would be the nifi-dbcp-service NAR classloader which doesn't see the snowflake driver as that is included in the nifi-snowflake-services NAR.
   
   As for the ```onConfigured()``` I see no new argument.
   I understand that there is a level of duplication but I think it's fine for the reasons stated before.
   Here's a bit more detailed explanation.
   
   If I keep the new propertyDescriptors (to be able to have separate ```name``` and ```displayName``` for them) then the shared code would require a refactor that would introduce the same amount of complexity we try to prevent with the refactor in the first place.
   I could create a method something like ```setupDatasource``` but it would get a bunch of parameters or would get a DTO which I would need to call getters and setters on - just like on the ```datasource``` itself.
   
   Again, it's not business logic but a sequence of getter and setter calls. No need to overengineer it.
   
   The case would be different if I didn't change the ```propertyDescriptors``` at all. In that case the ```setupDatasource``` could receive the ```context``` and the refactor would be clean.
   I don't like this direction because we would give up a higher level cleanness for a lower level one. But I can be convinced. If there are others as well who would vote for this approach I'll do the change.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r793828599



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       The Commons DBCP BasicDataSource does not need `DriverManager` when it has a `Driver` instance, and the `SnowflakeDriver` instance could be declared as a static variable.  The implied URL checking is useful, so that is an option.  Either approach to providing a simple method override would be good, as opposed to duplication.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1017835872


   Thanks for the background @joewitt, that is understandable, although it seems like the same argument could be made for any number of JDBC-based services.  The code duplication is a pain point, so I would be more favorable to this change if the implementation inherited from the standard DBCPConnectionPool service.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020451641


   I'm not sure this approach is what we want. The ```nifi-dbcp-shared``` would be basically ```nifi-dbcp-service``` itself.
   
   But more importantly, to my understanding we don't want to cut up modules and extract implementation code that can only be used by a select few modules in general.
   Unless the common code is very general and can be used by any module we want to maintain a fairly flat module hierarchy.
   If that is true, I don't see why this case would be an exception.
   But maybe I'm wrong.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020179087


   Thanks for following up on this @tpalfy.
   
   Having a NAR dependency can be problematic because it could pull in the META-INF directory with the Processor definition, which would result in the attempt to register the same processor more than once.  Excluding specific files from the destination NAR becomes problematic as well.
   
   It seems like the best approach would require refactoring shared code to a new module, and then depending on that module in both `nifi-dbcp-service` and `nifi-snowflake-service`.  Perhaps named something like `nifi-dbcp-shared`?  That module would contain the shared code, but would not include the META-INF directory.  That type of approach should avoid potential issues that would result by attempting to depend on the existing DBCP JAR or NAR modules as currently defined.  That will require some refactoring, but it seems like the best approach when it comes to avoiding duplication.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1017811317


   It gets long but we might need to call these things 'SnowflakeComputing' since we're referring to a trademarked term and such.  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792871046



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")
+public class SnowflakeConnectionPoolTest {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";
+//    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com/?db=test_db";
+//    public static final String SNOWFLAKE_URL = "jdbc:snowflake://tm55946.us-east-2.aws.snowflakecomputing.com";

Review comment:
       These comment lines serve a purpose which comments are good for: help the reader.
   This shows in a simple way the various ways to set the url.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] tpalfy commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
tpalfy commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792866514



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/test/java/org/apache/nifi/snowflake/service/SnowflakeConnectionPoolTest.java
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.snowflake.service;
+
+import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
+import org.apache.nifi.processor.ProcessContext;
+import org.apache.nifi.processor.ProcessSessionFactory;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.util.TestRunner;
+import org.apache.nifi.util.TestRunners;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Disabled;
+import org.junit.jupiter.api.Test;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.Statement;
+
+/**
+ * Set the following constants:<p>
+ * SNOWFLAKE_URL<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * SNOWFLAKE_PASSWORD<p>
+ * TABLE_NAME<p>
+ */
+@Disabled("Manual test.")
+public class SnowflakeConnectionPoolTest {
+    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com";
+//    public static final String SNOWFLAKE_URL = "tm55946.us-east-2.aws.snowflakecomputing.com/?db=test_db";
+//    public static final String SNOWFLAKE_URL = "jdbc:snowflake://tm55946.us-east-2.aws.snowflakecomputing.com";
+    public static final String SNOWFLAKE_USER = "???";
+    public static final String SNOWFLAKE_PASSWORD = "???";
+    public static final String TABLE_NAME = "test_db.public.test_table";
+
+    private static final String SERVICE_ID = SnowflakeComputingConnectionPool.class.getName();
+
+    private TestRunner runner;
+
+    private SnowflakeComputingConnectionPool service;
+
+    @Test
+    void testReadSnowflakeTable() throws Exception {
+        runner.assertValid(service);
+        runner.enableControllerService(service);
+        runner.assertValid(service);
+
+        try (
+            final Connection connection = service.getConnection();
+            final Statement st = connection.createStatement()
+        ) {
+            ResultSet resultSet = st.executeQuery("select * from " + TABLE_NAME);
+
+            int nrOfRows = 0;
+            while (resultSet.next()) {
+                nrOfRows++;
+            }
+
+            System.out.println("Read " + nrOfRows + " records from " + TABLE_NAME);

Review comment:
       This is a manual test where the exact configuration is up to the user.
   We can't decide the expected value.
   The purpose is not to test a full-fledged use-case but to test the connection.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] pvillard31 commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
pvillard31 commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1032805239


   @tpalfy - while I understand the willingness to not overcomplicate the code in the parent classes, I believe this is worth the effort as Snowflake may not be the only implementation we'll be seeing in the future. We can definitely think about additional implementations that would make things easier for NiFi users when interacting with SaaS-based databases. Can we keep as much as possible in the parent objects? I do share the concern about modifying and complicating the parent code but I think we have to if looking at this on the long run. Thoughts?
   
   Happy to hear additional thoughts though.
   @joewitt - you did comment this PR, how do you feel about it?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1042574240


   Thanks adding the license and updating the notice information @tpalfy. With those changes, this looks just about complete.
   
   Two recent updates for Commons Lang3 (#5773) and Commons DBCP (#5763) standardized the versioning of those dependencies and removed the need to declare those versions explicitly. I can update the branch to double-check the build if there are no other changes necessary.
   
   Do you have any additional feedback @turcsanyip? Otherwise I make the dependency adjustments to confirm a successful build and then merge.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] joewitt commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
joewitt commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1017871073


   Ahh yeah sorry fair enough.  I was really talking about the overall.  I totally agree we should minimize duplication as close to zero as possible.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1021271665


   Thanks @turcsanyip, that sounds like a good solution!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#issuecomment-1020481969


   There is a good deal of code duplication in certain places right now, so this is not necessarily so different, but it is an opportunity to find a better approach.
   
   The idea of a `nifi-dbcp-service-meta` sounds a good potential solution. I agree this does not follow the usual pattern of a utility module, so other approaches are certainly worth considering.  Moving the META-INF/services definition to a separate module would allow a `nifi-snowflake-service` module to depend on `nifi-dbcp-service`, so that sounds like a workable solution.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r793802743



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       I was able to configure and enable the `SnowflakeComputingConnectionPool` service and attempting a query using `ExecuteSQL` without any incompatible errors.
   
   As an even better solution, instead of adding `cloneAncestorResources = true`, overriding the `getDriver()` method to return a new instance of the `SnowflakeDriver` class also appears to work, since it avoids the `Class.forName()` lookup in `DBCPConnectionPool`.  The method in the SnowflakeComputing class would read as follows:
   
   ```suggestion
       @Override
       protected Driver getDriver(final String driverName, final String url) {
           return new SnowflakeDriver();
       }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [nifi] exceptionfactory commented on a change in pull request #5692: NIFI-9609 Added nifi-snowflake-bundle with a SnowflakeConnectionPool.

Posted by GitBox <gi...@apache.org>.
exceptionfactory commented on a change in pull request #5692:
URL: https://github.com/apache/nifi/pull/5692#discussion_r792889162



##########
File path: nifi-nar-bundles/nifi-snowflake-bundle/nifi-snowflake-services/src/main/java/org/apache/nifi/snowflake/service/SnowflakeComputingConnectionPool.java
##########
@@ -0,0 +1,314 @@
+/*
+ * 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.snowflake.service;
+
+import net.snowflake.client.jdbc.SnowflakeDriver;
+import org.apache.commons.dbcp2.BasicDataSource;
+import org.apache.commons.lang3.StringUtils;
+import org.apache.nifi.annotation.behavior.DynamicProperties;
+import org.apache.nifi.annotation.behavior.DynamicProperty;
+import org.apache.nifi.annotation.behavior.RequiresInstanceClassLoading;
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.annotation.lifecycle.OnDisabled;
+import org.apache.nifi.annotation.lifecycle.OnEnabled;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.components.PropertyValue;
+import org.apache.nifi.components.ValidationContext;
+import org.apache.nifi.components.ValidationResult;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPConnectionPool;
+import org.apache.nifi.expression.AttributeExpression;
+import org.apache.nifi.expression.ExpressionLanguageScope;
+import org.apache.nifi.processor.exception.ProcessException;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.reporting.InitializationException;
+
+import java.sql.Connection;
+import java.sql.Driver;
+import java.sql.DriverManager;
+import java.sql.SQLException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Implementation of Database Connection Pooling Service for Snowflake.
+ * Apache DBCP is used for connection pooling functionality.
+ */
+@Tags({"snowflake", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
+@CapabilityDescription("Provides Snowflake Connection Pooling Service. Connections can be asked from pool and returned after usage.")
+@DynamicProperties({
+    @DynamicProperty(name = "JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.VARIABLE_REGISTRY,
+        description = "Snowflake JDBC driver property name and value applied to JDBC connections."),
+    @DynamicProperty(name = "SENSITIVE.JDBC property name",
+        value = "Snowflake JDBC property value",
+        expressionLanguageScope = ExpressionLanguageScope.NONE,
+        description = "Snowflake JDBC driver property name prefixed with 'SENSITIVE.' handled as a sensitive property.")
+})
+@RequiresInstanceClassLoading
+public class SnowflakeComputingConnectionPool extends DBCPConnectionPool {
+
+    public static final PropertyDescriptor SNOWFLAKE_URL = new PropertyDescriptor.Builder()
+        .displayName("Snowflake URL")
+        .name("snowflake-url")
+        .description("E.g. 'cb56215.europe-west2.gcp.snowflakecomputing.com/?db=MY_DB'." +
+            " The '/?db=MY_DB' part can can have other connection parameters as well." +
+            " It can also be omitted but in that case tables need to be referenced with fully qualified names e.g. 'MY_DB.PUBLIC.MY_TABLe'.")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .required(true)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_USER = new PropertyDescriptor.Builder()
+        .displayName("Snowflake User Name")
+        .name("snowflake-user")
+        .defaultValue(null)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor SNOWFLAKE_PASSWORD = new PropertyDescriptor.Builder()
+        .displayName("Snowflake Password")
+        .name("snowflake-password")
+        .defaultValue(null)
+        .required(false)
+        .sensitive(true)
+        .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+        .expressionLanguageSupported(ExpressionLanguageScope.VARIABLE_REGISTRY)
+        .build();
+
+    public static final PropertyDescriptor VALIDATION_QUERY = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.VALIDATION_QUERY)
+        .displayName("Validation query")
+        .name("validation-query")
+        .build();
+
+    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_WAIT_TIME)
+        .displayName("Max Wait Time")
+        .name("max-wait-time")
+        .build();
+
+    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_TOTAL_CONNECTIONS)
+        .displayName("Max Total Connections")
+        .name("max-total-connections")
+        .build();
+
+    public static final PropertyDescriptor MIN_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_IDLE)
+        .displayName("Minimum Idle Connections")
+        .name("snowflake-min-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_IDLE = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_IDLE)
+        .displayName("Max Idle Connections")
+        .name("snowflake-max-idle-conns")
+        .build();
+
+    public static final PropertyDescriptor MAX_CONN_LIFETIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MAX_CONN_LIFETIME)
+        .displayName("Max Connection Lifetime")
+        .name("snowflake-max-conn-lifetime")
+        .build();
+
+    public static final PropertyDescriptor EVICTION_RUN_PERIOD = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.EVICTION_RUN_PERIOD)
+        .displayName("Time Between Eviction Runs")
+        .name("snowflake-time-between-eviction-runs")
+        .build();
+
+    public static final PropertyDescriptor MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Minimum Evictable Idle Time")
+        .name("snowflake-min-evictable-idle-time")
+        .build();
+
+    public static final PropertyDescriptor SOFT_MIN_EVICTABLE_IDLE_TIME = new PropertyDescriptor.Builder()
+        .fromPropertyDescriptor(DBCPConnectionPool.SOFT_MIN_EVICTABLE_IDLE_TIME)
+        .displayName("Soft Minimum Evictable Idle Time")
+        .name("snowflake-soft-min-evictable-idle-time")
+        .build();
+
+    private static final List<PropertyDescriptor> properties;
+
+    static {
+        final List<PropertyDescriptor> props = new ArrayList<>();
+        props.add(SNOWFLAKE_URL);
+        props.add(SNOWFLAKE_USER);
+        props.add(SNOWFLAKE_PASSWORD);
+        props.add(VALIDATION_QUERY);
+        props.add(MAX_WAIT_TIME);
+        props.add(MAX_TOTAL_CONNECTIONS);
+        props.add(MIN_IDLE);
+        props.add(MAX_IDLE);
+        props.add(MAX_CONN_LIFETIME);
+        props.add(EVICTION_RUN_PERIOD);
+        props.add(MIN_EVICTABLE_IDLE_TIME);
+        props.add(SOFT_MIN_EVICTABLE_IDLE_TIME);
+
+        properties = Collections.unmodifiableList(props);
+    }
+
+    private volatile BasicDataSource dataSource;
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    protected Collection<ValidationResult> customValidate(ValidationContext context) {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Configures connection pool by creating an instance of the
+     * {@link BasicDataSource} based on configuration provided with
+     * {@link ConfigurationContext}.
+     * <p>
+     * This operation makes no guarantees that the actual connection could be
+     * made since the underlying system may still go off-line during normal
+     * operation of the connection pool.
+     *
+     * @param context the configuration context
+     * @throws InitializationException if unable to create a database connection
+     */
+    @OnEnabled
+    public void onConfigured(final ConfigurationContext context) throws InitializationException {
+        final String snowflakeUrl = context.getProperty(SNOWFLAKE_URL).evaluateAttributeExpressions().getValue();
+        final String connectionString;
+        if (snowflakeUrl.startsWith("jdbc:snowflake")) {
+            connectionString = snowflakeUrl;
+        } else {
+            connectionString = "jdbc:snowflake://" + snowflakeUrl;
+        }
+        final String user = context.getProperty(SNOWFLAKE_USER).evaluateAttributeExpressions().getValue();
+        final String password = context.getProperty(SNOWFLAKE_PASSWORD).evaluateAttributeExpressions().getValue();
+
+        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).evaluateAttributeExpressions().asInteger();
+        final String validationQuery = context.getProperty(VALIDATION_QUERY).evaluateAttributeExpressions().getValue();
+        final Long maxWaitMillis = extractMillisWithInfinite(context.getProperty(MAX_WAIT_TIME).evaluateAttributeExpressions());
+        final Integer minIdle = context.getProperty(MIN_IDLE).evaluateAttributeExpressions().asInteger();
+        final Integer maxIdle = context.getProperty(MAX_IDLE).evaluateAttributeExpressions().asInteger();
+        final Long maxConnLifetimeMillis = extractMillisWithInfinite(context.getProperty(MAX_CONN_LIFETIME).evaluateAttributeExpressions());
+        final Long timeBetweenEvictionRunsMillis = extractMillisWithInfinite(context.getProperty(EVICTION_RUN_PERIOD).evaluateAttributeExpressions());
+        final Long minEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+        final Long softMinEvictableIdleTimeMillis = extractMillisWithInfinite(context.getProperty(SOFT_MIN_EVICTABLE_IDLE_TIME).evaluateAttributeExpressions());
+
+        dataSource = new BasicDataSource();
+
+        dataSource.setDriver(getDriver(SnowflakeDriver.class.getName(), connectionString));
+
+        dataSource.setUrl(connectionString);
+        dataSource.setUsername(user);
+        dataSource.setPassword(password);
+
+        dataSource.setMaxWaitMillis(maxWaitMillis);
+        dataSource.setMaxTotal(maxTotal);
+        dataSource.setMinIdle(minIdle);
+        dataSource.setMaxIdle(maxIdle);
+        dataSource.setMaxConnLifetimeMillis(maxConnLifetimeMillis);
+        dataSource.setTimeBetweenEvictionRunsMillis(timeBetweenEvictionRunsMillis);
+        dataSource.setMinEvictableIdleTimeMillis(minEvictableIdleTimeMillis);
+        dataSource.setSoftMinEvictableIdleTimeMillis(softMinEvictableIdleTimeMillis);
+
+        if (validationQuery != null && !validationQuery.isEmpty()) {
+            dataSource.setValidationQuery(validationQuery);
+            dataSource.setTestOnBorrow(true);
+        }
+
+        final List<PropertyDescriptor> dynamicProperties = context.getProperties()
+            .keySet()
+            .stream()
+            .filter(PropertyDescriptor::isDynamic)
+            .collect(Collectors.toList());
+
+        dynamicProperties.forEach((descriptor) -> {
+            final PropertyValue propertyValue = context.getProperty(descriptor);
+            if (descriptor.isSensitive()) {
+                final String propertyName = StringUtils.substringAfter(descriptor.getName(), SENSITIVE_PROPERTY_PREFIX);
+                dataSource.addConnectionProperty(propertyName, propertyValue.getValue());
+            } else {
+                dataSource.addConnectionProperty(descriptor.getName(), propertyValue.evaluateAttributeExpressions().getValue());
+            }
+        });
+    }
+
+    protected Driver getDriver(final String driverName, final String url) {

Review comment:
       Are you referring to exposing the `DBCPConnectionPool` to the list of available Controller Services through the `nifi-snowflake-services-nar`? I did not observe that behavior in runtime testing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@nifi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org