Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/261#discussion_r55759929
  
    --- Diff: 
nifi-nar-bundles/nifi-standard-bundle/nifi-standard-processors/src/main/java/org/apache/nifi/processors/standard/QueryDatabaseTable.java
 ---
    @@ -0,0 +1,607 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +package org.apache.nifi.processors.standard;
    +
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.Stateful;
    +import org.apache.nifi.annotation.behavior.WritesAttribute;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.state.Scope;
    +import org.apache.nifi.components.state.StateManager;
    +import org.apache.nifi.components.state.StateMap;
    +import org.apache.nifi.dbcp.DBCPService;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.processor.io.OutputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.processors.standard.util.JdbcCommon;
    +import org.apache.nifi.util.LongHolder;
    +import org.apache.nifi.util.StopWatch;
    +
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.math.BigDecimal;
    +import java.sql.Connection;
    +import java.sql.Date;
    +import java.sql.ResultSet;
    +import java.sql.ResultSetMetaData;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.sql.Timestamp;
    +import java.text.DecimalFormat;
    +import java.text.ParseException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashMap;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +import static java.sql.Types.ARRAY;
    +import static java.sql.Types.BIGINT;
    +import static java.sql.Types.BINARY;
    +import static java.sql.Types.BIT;
    +import static java.sql.Types.BLOB;
    +import static java.sql.Types.BOOLEAN;
    +import static java.sql.Types.CHAR;
    +import static java.sql.Types.CLOB;
    +import static java.sql.Types.DATE;
    +import static java.sql.Types.DECIMAL;
    +import static java.sql.Types.DOUBLE;
    +import static java.sql.Types.FLOAT;
    +import static java.sql.Types.INTEGER;
    +import static java.sql.Types.LONGNVARCHAR;
    +import static java.sql.Types.LONGVARBINARY;
    +import static java.sql.Types.LONGVARCHAR;
    +import static java.sql.Types.NCHAR;
    +import static java.sql.Types.NUMERIC;
    +import static java.sql.Types.NVARCHAR;
    +import static java.sql.Types.REAL;
    +import static java.sql.Types.ROWID;
    +import static java.sql.Types.SMALLINT;
    +import static java.sql.Types.TIME;
    +import static java.sql.Types.TIMESTAMP;
    +import static java.sql.Types.TINYINT;
    +import static java.sql.Types.VARBINARY;
    +import static java.sql.Types.VARCHAR;
    +
    +@EventDriven
    +@InputRequirement(Requirement.INPUT_ALLOWED)
    +@Tags({"sql", "select", "jdbc", "query", "database"})
    +@CapabilityDescription("Execute provided SQL select query. Query result 
will be converted to Avro format."
    +        + " Streaming is used so arbitrarily large result sets are 
supported. This processor can be scheduled to run on "
    +        + "a timer, or cron expression, using the standard scheduling 
methods, or it can be triggered by an incoming FlowFile. "
    +        + "If it is triggered by an incoming FlowFile, then attributes of 
that FlowFile will be available when evaluating the "
    +        + "select query. FlowFile attribute 'querydbtable.row.count' 
indicates how many rows were selected.")
    +@Stateful(scopes = Scope.CLUSTER, description = "After performing a query 
on the specified table, the maximum values for "
    +        + "the specified column(s) will be retained for use in future 
executions of the query. This allows the Processor "
    +        + "to fetch only those records that have max values greater than 
the retained values. This can be used for "
    +        + "incremental fetching, fetching of newly added rows, etc. To 
clear the maximum values, clear the state of the processor "
    +        + "per the State Management documentation")
    +@WritesAttribute(attribute = "querydbtable.row.count")
    +public class QueryDatabaseTable extends AbstractProcessor {
    +
    +    public static final String RESULT_ROW_COUNT = "querydbtable.row.count";
    +
    +    // Relationships
    +    public static final Relationship REL_SUCCESS = new 
Relationship.Builder()
    +            .name("success")
    +            .description("Successfully created FlowFile from SQL query 
result set.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new 
Relationship.Builder()
    +            .name("failure")
    +            .description("SQL query execution failed. Incoming FlowFile 
will be penalized and routed to this relationship.")
    +            .build();
    +    private final Set<Relationship> relationships;
    +
    +    public static final PropertyDescriptor DBCP_SERVICE = new 
PropertyDescriptor.Builder()
    +            .name("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 TABLE_NAME = new 
PropertyDescriptor.Builder()
    +            .name("Table Name")
    +            .description("The name of the database table to be queried.")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor COLUMN_NAMES = new 
PropertyDescriptor.Builder()
    +            .name("Columns to return")
    +            .description("A comma-separated list of column names to be 
used in the query. If your database requires "
    +                    + "special treatment of the names (quoting, e.g.), 
each name should include such treatment. If no "
    +                    + "column names are supplied, all columns in the 
specified table will be returned.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_VALUE_COLUMN_NAMES = new 
PropertyDescriptor.Builder()
    +            .name("Maximum-value Columns")
    +            .description("A comma-separated list of column names. The 
processor will keep track of the maximum value "
    +                    + "for each column that has been returned since the 
processor started running. This can be used to "
    +                    + "retrieve only those rows that have been 
added/updated since the last retrieval. Note that some "
    +                    + "JDBC types such as bit/boolean are not conducive to 
maintaining maximum value, so columns of these "
    +                    + "types should not be listed in this property, and 
will result in error(s) during processing.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor QUERY_TIMEOUT = new 
PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running 
SQL select query "
    +                    + ", zero means there is no limit. Max time less than 
1 second will be equal to zero.")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor SQL_PREPROCESS_STRATEGY = new 
PropertyDescriptor.Builder()
    +            .name("SQL Pre-processing Strategy")
    +            .description("The strategy to employ when generating the SQL 
for querying the table. A strategy may include "
    +                    + "custom or database-specific code, such as the 
treatment of time/date formats.")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .allowableValues("None", "Oracle")
    +            .defaultValue("None")
    +            .build();
    +
    +
    +    private final List<PropertyDescriptor> propDescriptors;
    +
    +    protected final Map<String, Integer> columnTypeMap = new HashMap<>();
    +
    +    public QueryDatabaseTable() {
    +        final Set<Relationship> r = new HashSet<>();
    +        r.add(REL_SUCCESS);
    +        r.add(REL_FAILURE);
    +        relationships = Collections.unmodifiableSet(r);
    +
    +        final List<PropertyDescriptor> pds = new ArrayList<>();
    +        pds.add(DBCP_SERVICE);
    +        pds.add(TABLE_NAME);
    +        pds.add(COLUMN_NAMES);
    +        pds.add(MAX_VALUE_COLUMN_NAMES);
    +        pds.add(QUERY_TIMEOUT);
    +        pds.add(SQL_PREPROCESS_STRATEGY);
    +        propDescriptors = Collections.unmodifiableList(pds);
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return propDescriptors;
    +    }
    +
    +    @OnScheduled
    +    public void setup(final ProcessContext context) {
    +        // Try to fill the columnTypeMap with the types of the desired 
max-value columns
    +        final DBCPService dbcpService = 
context.getProperty(DBCP_SERVICE).asControllerService(DBCPService.class);
    +        final String tableName = 
context.getProperty(TABLE_NAME).getValue();
    +        final String maxValueColumnNames = 
context.getProperty(MAX_VALUE_COLUMN_NAMES).getValue();
    +
    +        try (final Connection con = dbcpService.getConnection();
    +             final Statement st = con.createStatement()) {
    +
    +            // Try a query that returns no rows, for the purposes of 
getting metadata about the columns. It is possible
    +            // to use DatabaseMetaData.getColumns(), but not all drivers 
support this, notably the schema-on-read
    +            // approach as in Apache Drill
    +            String query = getSelectFromClause(tableName, 
maxValueColumnNames).append(" WHERE 1 = 0").toString();
    +            ResultSet resultSet = st.executeQuery(query);
    +            ResultSetMetaData resultSetMetaData = resultSet.getMetaData();
    +            int numCols = resultSetMetaData.getColumnCount();
    +            if (numCols > 0) {
    +                columnTypeMap.clear();
    +                for (int i = 1; i <= numCols; i++) {
    +                    String colName = 
resultSetMetaData.getColumnName(i).toLowerCase();
    +                    int colType = resultSetMetaData.getColumnType(i);
    +                    columnTypeMap.put(colName, colType);
    +                }
    +
    +            } else {
    +                throw new ProcessException("No columns found in table from 
those specified: " + maxValueColumnNames);
    +            }
    +
    +        } catch (SQLException e) {
    +            throw new ProcessException(e);
    --- End diff --
    
    Why catch the SQLException and wrap it? Would recommend just allowing the 
SQLException to fly -- or, better yet, throw a ProcessException that wraps 
SQLException but provides more information about what's going on, such as: new 
ProcessException("Unable to communicate with database in order to determine 
column types", e);


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to