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/09/10 13:51:40 UTC

[GitHub] [nifi] gresockj opened a new pull request, #6391: NIFI-9402: Adding DatabaseParameterProvider

gresockj opened a new pull request, #6391:
URL: https://github.com/apache/nifi/pull/6391

   
   # Summary
   
   [NIFI-9402](https://issues.apache.org/jira/browse/NIFI-9402)
   
   # Tracking
   
   Please complete the following tracking steps prior to pull request creation.
   
   ### Issue Tracking
   
   - [ ] [Apache NiFi Jira](https://issues.apache.org/jira/browse/NIFI) issue created
   
   ### Pull Request Tracking
   
   - [ ] Pull Request title starts with Apache NiFi Jira issue number, such as `NIFI-00000`
   - [ ] Pull Request commit message starts with Apache NiFi Jira issue number, as such `NIFI-00000`
   
   ### Pull Request Formatting
   
   - [ ] Pull Request based on current revision of the `main` branch
   - [ ] Pull Request refers to a feature branch with one commit containing changes
   
   # Verification
   
   Please indicate the verification steps performed prior to pull request creation.
   
   ### Build
   
   - [ ] Build completed using `mvn clean install -P contrib-check`
     - [ ] JDK 8
     - [ ] JDK 11
     - [ ] JDK 17
   
   ### Licensing
   
   - [ ] New dependencies are compatible with the [Apache License 2.0](https://apache.org/licenses/LICENSE-2.0) according to the [License Policy](https://www.apache.org/legal/resolved.html)
   - [ ] New dependencies are documented in applicable `LICENSE` and `NOTICE` files
   
   ### Documentation
   
   - [ ] Documentation formatting appears as expected in rendered files
   


-- 
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] mattyb149 commented on a diff in pull request #6391: NIFI-9402: Adding DatabaseParameterProvider

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on code in PR #6391:
URL: https://github.com/apache/nifi/pull/6391#discussion_r981874236


##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/resources/docs/org.apache.nifi.parameter.DatabaseParameterProvider/additionalDetails.html:
##########
@@ -0,0 +1,187 @@
+<!DOCTYPE html>
+<html lang="en" xmlns="http://www.w3.org/1999/html">
+<!--
+      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.
+    -->
+
+<head>
+    <meta charset="utf-8"/>
+    <title>DatabaseParameterProvider</title>
+    <link rel="stylesheet" href="../../../../../css/component-usage.css" type="text/css"/>
+</head>
+<body>
+
+<h1>Providing Parameters from a Database</h1>
+
+<p>
+    The DatabaseParameterProvider at its core maps database rows to Parameters, specified by a
+    Parameter Name Column and Parameter Value Column.  The Parameter Group name must also be accounted for, and may
+    be specified in different ways using the Parameter Grouping Strategy.
+</p>
+
+<p>
+    The default configuration uses a fully column-based approach, with the Parameter Group Name
+    also specified by columns in the same table.  An example of a table using this configuration would be:
+</p>
+<table>
+    <thead>
+        <tr>
+            <th colspan="4" style="text-align: center">PARAMETER_CONTEXTS</th>
+        </tr>
+        <tr>
+            <th>PARAMETER_NAME</th><th>PARAMETER_VALUE</th><th>PARAMETER_GROUP</th>
+        </tr>
+    </thead>
+    <tbody>
+        <tr>
+            <td>param.foo</td><td>value-foo</td><td>group_1</td>
+        </tr>
+        <tr>
+            <td>param.bar</td><td>value-bar</td><td>group_1</td>
+        </tr>
+        <tr>
+            <td>param.one</td><td>value-one</td><td>group_2</td>
+        </tr>
+        <tr>
+            <td>param.two</td><td>value-two</td><td>group_2</td>
+        </tr>
+    </tbody>
+    <caption>Table 1: Database table example with Grouping Strategy = Column</caption>
+</table>
+
+<p>
+    In order to use the data from this table, set the following Properties:
+</p>
+
+<ul>
+    <li><b>Parameter Grouping Strategy</b> - Column</li>
+    <li><b>Table Name</b> - PARAMETER_CONTEXTS</li>
+    <li><b>Parameter Name Column</b> - PARAMETER_NAME</li>
+    <li><b>Parameter Value Column</b> - PARAMETER_VALUE</li>
+    <li><b>Parameter Group Name Column</b> - PARAMETER_GROUP</li>
+</ul>
+
+<p>
+    Note: in some databases, the words 'PARAMETER', 'PARAMETERS', 'GROUP', and even 'VALUE' are reserved words.

Review Comment:
   Maybe add to this to check the database docs and/or quote the words per the DB doc



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/java/org/apache/nifi/parameter/DatabaseParameterProvider.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.parameter;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.db.DatabaseAdapter;
+import org.apache.nifi.util.StringUtils;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.ServiceLoader;
+import java.util.stream.Collectors;
+
+@Tags({"database", "dbcp", "sql"})
+@CapabilityDescription("Fetches parameters from database tables")
+
+public class DatabaseParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+
+    protected final static Map<String, DatabaseAdapter> dbAdapters = new HashMap<>();
+
+    public static final PropertyDescriptor DB_TYPE;
+
+    static {
+        // Load the DatabaseAdapters
+        ArrayList<AllowableValue> dbAdapterValues = new ArrayList<>();
+        ServiceLoader<DatabaseAdapter> dbAdapterLoader = ServiceLoader.load(DatabaseAdapter.class);
+        dbAdapterLoader.forEach(it -> {
+            dbAdapters.put(it.getName(), it);
+            dbAdapterValues.add(new AllowableValue(it.getName(), it.getName(), it.getDescription()));
+        });
+
+        DB_TYPE = new PropertyDescriptor.Builder()
+                .name("db-type")
+                .displayName("Database Type")
+                .description("The type/flavor of database, used for generating database-specific code. In many cases the Generic type "
+                        + "should suffice, but some databases (such as Oracle) require custom SQL clauses. ")
+                .allowableValues(dbAdapterValues.toArray(new AllowableValue[dbAdapterValues.size()]))
+                .defaultValue("Generic")
+                .required(true)
+                .build();
+    }
+
+    static AllowableValue GROUPING_BY_COLUMN = new AllowableValue("grouping-by-column", "Column",
+            "A single table is partitioned by the 'Parameter Group Name Column'.  All rows with the same value in this column will " +
+                    "map to a group of the same name.");
+    static AllowableValue GROUPING_BY_TABLE_NAME = new AllowableValue("grouping-by-table-name", "Table Name",
+            "An entire table maps to a Parameter Group.  The group name will be the table name.");
+
+    public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder()
+            .name("dbcp-service")
+            .displayName("Database Connection Pooling Service")
+            .description("The Controller Service that is used to obtain a connection to the database.")
+            .required(true)
+            .identifiesControllerService(DBCPService.class)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_GROUPING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("parameter-grouping-strategy")
+            .displayName("Parameter Grouping Strategy")
+            .description("The strategy used to group parameters.")
+            .required(true)
+            .allowableValues(new AllowableValue[] { GROUPING_BY_COLUMN, GROUPING_BY_TABLE_NAME })
+            .defaultValue(GROUPING_BY_COLUMN.getValue())
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAMES = new PropertyDescriptor.Builder()
+            .name("table-names")
+            .displayName("Table Names")
+            .description("A comma-separated list of names of the database tables containing the parameters.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_TABLE_NAME)
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("table-name")
+            .displayName("Table Name")
+            .description("The name of the database table containing the parameters.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_COLUMN)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_NAME_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-name-column")
+            .displayName("Parameter Name Column")
+            .description("The name of a column containing the parameter name.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_VALUE_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-value-column")
+            .displayName("Parameter Value Column")
+            .description("The name of a column containing the parameter value.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_GROUP_NAME_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-group-name-column")
+            .displayName("Parameter Group Name Column")
+            .description("The name of a column containing the name of the parameter group into which the parameter should be mapped.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_COLUMN)
+            .build();
+
+    public static final PropertyDescriptor SQL_WHERE_CLAUSE = new PropertyDescriptor.Builder()

Review Comment:
   Should we add an example in the additionalDetails for this?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/java/org/apache/nifi/parameter/DatabaseParameterProvider.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.parameter;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.db.DatabaseAdapter;
+import org.apache.nifi.util.StringUtils;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.ServiceLoader;
+import java.util.stream.Collectors;
+
+@Tags({"database", "dbcp", "sql"})
+@CapabilityDescription("Fetches parameters from database tables")
+
+public class DatabaseParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+
+    protected final static Map<String, DatabaseAdapter> dbAdapters = new HashMap<>();
+
+    public static final PropertyDescriptor DB_TYPE;
+
+    static {
+        // Load the DatabaseAdapters
+        ArrayList<AllowableValue> dbAdapterValues = new ArrayList<>();
+        ServiceLoader<DatabaseAdapter> dbAdapterLoader = ServiceLoader.load(DatabaseAdapter.class);
+        dbAdapterLoader.forEach(it -> {
+            dbAdapters.put(it.getName(), it);
+            dbAdapterValues.add(new AllowableValue(it.getName(), it.getName(), it.getDescription()));
+        });
+
+        DB_TYPE = new PropertyDescriptor.Builder()
+                .name("db-type")
+                .displayName("Database Type")
+                .description("The type/flavor of database, used for generating database-specific code. In many cases the Generic type "
+                        + "should suffice, but some databases (such as Oracle) require custom SQL clauses. ")
+                .allowableValues(dbAdapterValues.toArray(new AllowableValue[dbAdapterValues.size()]))
+                .defaultValue("Generic")
+                .required(true)
+                .build();
+    }
+
+    static AllowableValue GROUPING_BY_COLUMN = new AllowableValue("grouping-by-column", "Column",
+            "A single table is partitioned by the 'Parameter Group Name Column'.  All rows with the same value in this column will " +
+                    "map to a group of the same name.");
+    static AllowableValue GROUPING_BY_TABLE_NAME = new AllowableValue("grouping-by-table-name", "Table Name",
+            "An entire table maps to a Parameter Group.  The group name will be the table name.");
+
+    public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder()
+            .name("dbcp-service")
+            .displayName("Database Connection Pooling Service")
+            .description("The Controller Service that is used to obtain a connection to the database.")
+            .required(true)
+            .identifiesControllerService(DBCPService.class)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_GROUPING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("parameter-grouping-strategy")
+            .displayName("Parameter Grouping Strategy")
+            .description("The strategy used to group parameters.")
+            .required(true)
+            .allowableValues(new AllowableValue[] { GROUPING_BY_COLUMN, GROUPING_BY_TABLE_NAME })
+            .defaultValue(GROUPING_BY_COLUMN.getValue())
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAMES = new PropertyDescriptor.Builder()
+            .name("table-names")
+            .displayName("Table Names")
+            .description("A comma-separated list of names of the database tables containing the parameters.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_TABLE_NAME)
+            .build();
+
+    public static final PropertyDescriptor TABLE_NAME = new PropertyDescriptor.Builder()
+            .name("table-name")
+            .displayName("Table Name")
+            .description("The name of the database table containing the parameters.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_COLUMN)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_NAME_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-name-column")
+            .displayName("Parameter Name Column")
+            .description("The name of a column containing the parameter name.")
+            .required(true)
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_VALUE_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-value-column")
+            .displayName("Parameter Value Column")
+            .description("The name of a column containing the parameter value.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_GROUP_NAME_COLUMN = new PropertyDescriptor.Builder()
+            .name("parameter-group-name-column")
+            .displayName("Parameter Group Name Column")
+            .description("The name of a column containing the name of the parameter group into which the parameter should be mapped.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .required(true)
+            .dependsOn(PARAMETER_GROUPING_STRATEGY, GROUPING_BY_COLUMN)
+            .build();
+
+    public static final PropertyDescriptor SQL_WHERE_CLAUSE = new PropertyDescriptor.Builder()
+            .name("sql-where-clause")
+            .displayName("SQL WHERE clause")
+            .description("A optional SQL query 'WHERE' clause by which to filter all results.  The 'WHERE' keyword should not be included.")
+            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
+            .build();
+
+    private List<PropertyDescriptor> properties;
+
+    @Override
+    protected void init(final ParameterProviderInitializationContext config) {
+        final List<PropertyDescriptor> properties = new ArrayList<>();
+        properties.add(DB_TYPE);
+        properties.add(DBCP_SERVICE);
+        properties.add(PARAMETER_GROUPING_STRATEGY);
+        properties.add(TABLE_NAME);
+        properties.add(TABLE_NAMES);
+        properties.add(PARAMETER_NAME_COLUMN);
+        properties.add(PARAMETER_VALUE_COLUMN);
+        properties.add(PARAMETER_GROUP_NAME_COLUMN);
+        properties.add(SQL_WHERE_CLAUSE);
+
+        this.properties = Collections.unmodifiableList(properties);
+    }
+
+    @Override
+    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
+        return properties;
+    }
+
+    @Override
+    public List<ParameterGroup> fetchParameters(final ConfigurationContext context) {
+        final boolean groupByColumn = GROUPING_BY_COLUMN.getValue().equals(context.getProperty(PARAMETER_GROUPING_STRATEGY).getValue());
+
+        final DBCPService dbcpService = context.getProperty(DBCP_SERVICE).asControllerService(DBCPService.class);
+        final String whereClause = context.getProperty(SQL_WHERE_CLAUSE).getValue();
+        final String parameterNameColumn = context.getProperty(PARAMETER_NAME_COLUMN).getValue();
+        final String parameterValueColumn = context.getProperty(PARAMETER_VALUE_COLUMN).getValue();
+        final String parameterGroupNameColumn = context.getProperty(PARAMETER_GROUP_NAME_COLUMN).getValue();
+
+        final List<String> tableNames = groupByColumn
+                ? Collections.singletonList(context.getProperty(TABLE_NAME).getValue())
+                : Arrays.stream(context.getProperty(TABLE_NAMES).getValue().split(",")).map(String::trim).collect(Collectors.toList());
+
+        final Map<String, List<Parameter>> parameterMap = new HashMap<>();
+        for (final String tableName : tableNames) {
+            try (final Connection con = dbcpService.getConnection(Collections.emptyMap()); final Statement st = con.createStatement()) {
+                final List<String> columns = new ArrayList<>();
+                columns.add(parameterNameColumn);
+                columns.add(parameterValueColumn);
+                if (groupByColumn) {
+                    columns.add(parameterGroupNameColumn);
+                }
+                final String query = getQuery(context, tableName, columns, whereClause);
+
+                getLogger().info("Fetching parameters with query: " + query);
+                try (final ResultSet rs = st.executeQuery(query)) {
+                    while (rs.next()) {
+                        final String parameterName = Objects.requireNonNull(rs.getString(parameterNameColumn), "Parameter Name may not be null");
+                        final String parameterValue = Objects.requireNonNull(rs.getString(parameterValueColumn), "Parameter Value may not be null");
+                        final String parameterGroupName;
+                        if (groupByColumn) {
+                            parameterGroupName = parameterGroupNameColumn == null ? null : rs.getString(parameterGroupNameColumn);
+                        } else {
+                            parameterGroupName = tableName;
+                        }
+
+                        final ParameterDescriptor parameterDescriptor = new ParameterDescriptor.Builder()
+                                .name(parameterName)
+                                .build();
+                        final Parameter parameter = new Parameter(parameterDescriptor, parameterValue);
+                        parameterMap.computeIfAbsent(parameterGroupName, key -> new ArrayList<>()).add(parameter);
+                    }
+                }
+            } catch (final SQLException e) {
+                throw new RuntimeException("Encountered a database error when fetching parameters: " + e.getMessage(), e);

Review Comment:
   Is there a better way to surface this error? I get the "Unexpected error occurred" page and have to look in nifi-user.log to figure out what went wrong. Can we put up a bulletin or something in the UI for the user?



##########
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-parameter-providers/src/main/java/org/apache/nifi/parameter/DatabaseParameterProvider.java:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.parameter;
+
+import org.apache.nifi.annotation.documentation.CapabilityDescription;
+import org.apache.nifi.annotation.documentation.Tags;
+import org.apache.nifi.components.AllowableValue;
+import org.apache.nifi.components.ConfigVerificationResult;
+import org.apache.nifi.components.PropertyDescriptor;
+import org.apache.nifi.controller.ConfigurationContext;
+import org.apache.nifi.dbcp.DBCPService;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.processor.util.StandardValidators;
+import org.apache.nifi.processors.standard.db.DatabaseAdapter;
+import org.apache.nifi.util.StringUtils;
+
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.ServiceLoader;
+import java.util.stream.Collectors;
+
+@Tags({"database", "dbcp", "sql"})
+@CapabilityDescription("Fetches parameters from database tables")
+
+public class DatabaseParameterProvider extends AbstractParameterProvider implements VerifiableParameterProvider {
+
+    protected final static Map<String, DatabaseAdapter> dbAdapters = new HashMap<>();
+
+    public static final PropertyDescriptor DB_TYPE;
+
+    static {
+        // Load the DatabaseAdapters
+        ArrayList<AllowableValue> dbAdapterValues = new ArrayList<>();
+        ServiceLoader<DatabaseAdapter> dbAdapterLoader = ServiceLoader.load(DatabaseAdapter.class);
+        dbAdapterLoader.forEach(it -> {
+            dbAdapters.put(it.getName(), it);
+            dbAdapterValues.add(new AllowableValue(it.getName(), it.getName(), it.getDescription()));
+        });
+
+        DB_TYPE = new PropertyDescriptor.Builder()
+                .name("db-type")
+                .displayName("Database Type")
+                .description("The type/flavor of database, used for generating database-specific code. In many cases the Generic type "
+                        + "should suffice, but some databases (such as Oracle) require custom SQL clauses. ")
+                .allowableValues(dbAdapterValues.toArray(new AllowableValue[dbAdapterValues.size()]))
+                .defaultValue("Generic")
+                .required(true)
+                .build();
+    }
+
+    static AllowableValue GROUPING_BY_COLUMN = new AllowableValue("grouping-by-column", "Column",
+            "A single table is partitioned by the 'Parameter Group Name Column'.  All rows with the same value in this column will " +
+                    "map to a group of the same name.");
+    static AllowableValue GROUPING_BY_TABLE_NAME = new AllowableValue("grouping-by-table-name", "Table Name",
+            "An entire table maps to a Parameter Group.  The group name will be the table name.");
+
+    public static final PropertyDescriptor DBCP_SERVICE = new PropertyDescriptor.Builder()
+            .name("dbcp-service")
+            .displayName("Database Connection Pooling Service")
+            .description("The Controller Service that is used to obtain a connection to the database.")
+            .required(true)
+            .identifiesControllerService(DBCPService.class)
+            .build();
+
+    public static final PropertyDescriptor PARAMETER_GROUPING_STRATEGY = new PropertyDescriptor.Builder()
+            .name("parameter-grouping-strategy")
+            .displayName("Parameter Grouping Strategy")
+            .description("The strategy used to group parameters.")
+            .required(true)
+            .allowableValues(new AllowableValue[] { GROUPING_BY_COLUMN, GROUPING_BY_TABLE_NAME })

Review Comment:
   Nitpick, my IDE said you can just use the varargs signature rather than creating an array :)



-- 
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] mattyb149 closed pull request #6391: NIFI-9402: Adding DatabaseParameterProvider

Posted by GitBox <gi...@apache.org>.
mattyb149 closed pull request #6391: NIFI-9402: Adding DatabaseParameterProvider
URL: https://github.com/apache/nifi/pull/6391


-- 
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] mattyb149 commented on pull request #6391: NIFI-9402: Adding DatabaseParameterProvider

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on PR #6391:
URL: https://github.com/apache/nifi/pull/6391#issuecomment-1258943926

   Reviewing...


-- 
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] mattyb149 commented on pull request #6391: NIFI-9402: Adding DatabaseParameterProvider

Posted by GitBox <gi...@apache.org>.
mattyb149 commented on PR #6391:
URL: https://github.com/apache/nifi/pull/6391#issuecomment-1261498104

   +1 LGTM, changed line 206 to use the column name and fixed a Checkstyle violation on merge. Verified the provider works as expected in happy and non-happy paths. Thanks for the new feature! Merging 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