You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@nifi.apache.org by mattyb149 <gi...@git.apache.org> on 2016/04/20 20:44:42 UTC

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

GitHub user mattyb149 opened a pull request:

    https://github.com/apache/nifi/pull/372

    NIFI-981 [REVIEW-ONLY]: Initial addition of HiveQL processors

    I'd like to get some feedback on the ExecuteHiveQL and PutHiveQL processors before I start adding Kerberos support and such. I will incorporate any comments into the code and then re-issue a PR when the processors are ready for final review.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mattyb149/nifi hive-bundle

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/nifi/pull/372.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #372
    
----

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60468517
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java ---
    @@ -0,0 +1,320 @@
    +/*
    + * 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.hive;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.dbcp.hive.HiveDBCPService;
    +import org.apache.nifi.flowfile.FlowFile;
    +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.InputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.stream.io.StreamUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.math.BigDecimal;
    +import java.nio.charset.Charset;
    +import java.sql.Connection;
    +import java.sql.Date;
    +import java.sql.PreparedStatement;
    +import java.sql.SQLException;
    +import java.sql.SQLNonTransientException;
    +import java.sql.Time;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SeeAlso(ExecuteHiveQL.class)
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"sql", "hive", "put", "database", "update", "insert"})
    +@CapabilityDescription("Executes a HiveQL DDL/DML command (UPDATE, INSERT, e.g.). The content of an incoming FlowFile is expected to be the HiveQL command "
    +        + "to execute. The HiveQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
    +        + "with the naming convention hiveql.args.N.type and hiveql.args.N.value, where N is a positive integer. The hiveql.args.N.type is expected to be "
    +        + "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "hiveql.args.N.type", description = "Incoming FlowFiles are expected to be parameterized HiveQL statements. The type of each Parameter is specified as an integer "
    +                + "that represents the JDBC Type of the parameter."),
    +        @ReadsAttribute(attribute = "hiveql.args.N.value", description = "Incoming FlowFiles are expected to be parameterized HiveQL statements. The value of the Parameters are specified as "
    +                + "hiveql.args.1.value, hiveql.args.2.value, hiveql.args.3.value, and so on. The type of the hiveql.args.1.value Parameter is specified by the hiveql.args.1.type attribute.")
    +})
    +public class PutHiveQL extends AbstractHiveQLProcessor {
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The preferred number of FlowFiles to put to the database in a single transaction")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
    +            .name("Character Set")
    +            .description("Specifies the character set of the record data.")
    +            .required(true)
    +            .defaultValue("UTF-8")
    +            .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("A FlowFile is routed to this relationship after the database is successfully updated")
    +            .build();
    +    public static final Relationship REL_RETRY = new Relationship.Builder()
    +            .name("retry")
    +            .description("A FlowFile is routed to this relationship if the database cannot be updated but attempting the operation again may succeed")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("A FlowFile is routed to this relationship if the database cannot be updated and retrying the operation will also fail, "
    +                    + "such as an invalid query or an integrity constraint violation")
    +            .build();
    +
    +    private static final Pattern HIVEQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("hiveql\\.args\\.(\\d+)\\.type");
    +    private static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
    +
    +    private final static List<PropertyDescriptor> propertyDescriptors;
    +    private final static Set<Relationship> relationships;
    +
    +    /*
    +     * Will ensure that the list of property descriptors is build only once.
    +     * Will also create a Set of relationships
    +     */
    +    static {
    +        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
    +        _propertyDescriptors.add(HIVE_DBCP_SERVICE);
    +        _propertyDescriptors.add(BATCH_SIZE);
    +        _propertyDescriptors.add(CHARSET);
    +        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
    +
    +        Set<Relationship> _relationships = new HashSet<>();
    +        _relationships.add(REL_SUCCESS);
    +        _relationships.add(REL_FAILURE);
    +        _relationships.add(REL_RETRY);
    +        relationships = Collections.unmodifiableSet(_relationships);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return propertyDescriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        final List<FlowFile> flowFiles = session.get(batchSize);
    +
    +        if (flowFiles.isEmpty()) {
    +            return;
    +        }
    +
    +        final long startNanos = System.nanoTime();
    +        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
    +        final HiveDBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(HiveDBCPService.class);
    +        try (final Connection conn = dbcpService.getConnection()) {
    +
    +            for (FlowFile flowFile : flowFiles) {
    +                try {
    +                    final String hiveQL = getHiveQL(session, flowFile, charset);
    +                    final PreparedStatement stmt = conn.prepareStatement(hiveQL);
    +                    setParameters(stmt, flowFile.getAttributes());
    +
    +                    // Execute the statement
    +                    stmt.execute();
    +
    +                    // Determine the database URL from the connection metadata
    +                    String url = "jdbc:hive2://unknown-host";
    +                    try {
    +                        url = conn.getMetaData().getURL();
    +                    } catch (final SQLException sqle) {
    +                        // Just use the default
    +                    }
    +
    +                    // Emit a Provenance SEND event
    +                    final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
    +                    session.getProvenanceReporter().send(flowFile, url, transmissionMillis, true);
    +                    session.transfer(flowFile, REL_SUCCESS);
    +
    +                } catch (final SQLException e) {
    +
    +                    if (e instanceof SQLNonTransientException) {
    +                        getLogger().error("Failed to update Hive for {} due to {}; routing to failure", new Object[]{flowFile, e});
    +                        session.transfer(flowFile, REL_FAILURE);
    +                    } else {
    +                        getLogger().error("Failed to update Hive for {} due to {}; it is possible that retrying the operation will succeed, so routing to retry", new Object[]{flowFile, e});
    +                        flowFile = session.penalize(flowFile);
    +                        session.transfer(flowFile, REL_RETRY);
    +                    }
    +
    +                }
    +            }
    +        } catch (final SQLException sqle) {
    +            // There was a problem getting the connection, yield and retry the flowfiles
    +            getLogger().error("Failed to get Hive connection due to {}; it is possible that retrying the operation will succeed, so routing to retry", new Object[]{sqle});
    +            session.transfer(flowFiles, REL_RETRY);
    +            context.yield();
    +        }
    +    }
    +
    +    /**
    +     * Determines the HiveQL statement that should be executed for the given FlowFile
    +     *
    +     * @param session  the session that can be used to access the given FlowFile
    +     * @param flowFile the FlowFile whose HiveQL statement should be executed
    +     * @return the HiveQL that is associated with the given FlowFile
    +     */
    +    private String getHiveQL(final ProcessSession session, final FlowFile flowFile, final Charset charset) {
    +        // Read the HiveQL from the FlowFile's content
    +        final byte[] buffer = new byte[(int) flowFile.getSize()];
    +        session.read(flowFile, new InputStreamCallback() {
    +            @Override
    +            public void process(final InputStream in) throws IOException {
    +                StreamUtils.fillBuffer(in, buffer);
    +            }
    +        });
    +
    +        // Create the PreparedStatement to use for this FlowFile.
    +        return new String(buffer, charset);
    +    }
    +
    +
    +    /**
    +     * Sets all of the appropriate parameters on the given PreparedStatement, based on the given FlowFile attributes.
    +     *
    +     * @param stmt       the statement to set the parameters on
    +     * @param attributes the attributes from which to derive parameter indices, values, and types
    +     * @throws SQLException if the PreparedStatement throws a SQLException when the appropriate setter is called
    +     */
    +    private void setParameters(final PreparedStatement stmt, final Map<String, String> attributes) throws SQLException {
    +        for (final Map.Entry<String, String> entry : attributes.entrySet()) {
    +            final String key = entry.getKey();
    +            final Matcher matcher = HIVEQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
    +            if (matcher.matches()) {
    +                final int parameterIndex = Integer.parseInt(matcher.group(1));
    +
    +                final boolean isNumeric = NUMBER_PATTERN.matcher(entry.getValue()).matches();
    +                if (!isNumeric) {
    +                    throw new ProcessException("Value of the " + key + " attribute is '" + entry.getValue() + "', which is not a valid JDBC numeral type");
    +                }
    +
    +                final int jdbcType = Integer.parseInt(entry.getValue());
    +                final String valueAttrName = "hiveql.args." + parameterIndex + ".value";
    +                final String parameterValue = attributes.get(valueAttrName);
    +
    +                try {
    +                    setParameter(stmt, valueAttrName, parameterIndex, parameterValue, jdbcType);
    +                } catch (final NumberFormatException nfe) {
    +                    throw new ProcessException("The value of the " + valueAttrName + " is '" + parameterValue + "', which cannot be converted into the necessary data type", nfe);
    +                }
    +            }
    +        }
    +    }
    +
    +    /**
    +     * Determines how to map the given value to the appropriate JDBC data type and sets the parameter on the
    +     * provided PreparedStatement
    +     *
    +     * @param stmt           the PreparedStatement to set the parameter on
    +     * @param attrName       the name of the attribute that the parameter is coming from - for logging purposes
    +     * @param parameterIndex the index of the HiveQL parameter to set
    +     * @param parameterValue the value of the HiveQL parameter to set
    +     * @param jdbcType       the JDBC Type of the HiveQL parameter to set
    +     * @throws SQLException if the PreparedStatement throws a SQLException when calling the appropriate setter
    +     */
    +    private void setParameter(final PreparedStatement stmt, final String attrName, final int parameterIndex, final String parameterValue, final int jdbcType) throws SQLException {
    +        if (parameterValue == null) {
    +            stmt.setNull(parameterIndex, jdbcType);
    +        } else {
    +            try {
    +                switch (jdbcType) {
    +                    case Types.BIT:
    +                    case Types.BOOLEAN:
    +                        stmt.setBoolean(parameterIndex, Boolean.parseBoolean(parameterValue));
    +                        break;
    +                    case Types.TINYINT:
    +                        stmt.setByte(parameterIndex, Byte.parseByte(parameterValue));
    +                        break;
    +                    case Types.SMALLINT:
    +                        stmt.setShort(parameterIndex, Short.parseShort(parameterValue));
    +                        break;
    +                    case Types.INTEGER:
    +                        stmt.setInt(parameterIndex, Integer.parseInt(parameterValue));
    +                        break;
    +                    case Types.BIGINT:
    +                        stmt.setLong(parameterIndex, Long.parseLong(parameterValue));
    +                        break;
    +                    case Types.REAL:
    +                        stmt.setFloat(parameterIndex, Float.parseFloat(parameterValue));
    +                        break;
    +                    case Types.FLOAT:
    +                    case Types.DOUBLE:
    +                        stmt.setDouble(parameterIndex, Double.parseDouble(parameterValue));
    +                        break;
    +                    case Types.DECIMAL:
    +                    case Types.NUMERIC:
    +                        stmt.setBigDecimal(parameterIndex, new BigDecimal(parameterValue));
    +                        break;
    +                    case Types.DATE:
    +                        stmt.setDate(parameterIndex, new Date(Long.parseLong(parameterValue)));
    +                        break;
    +                    case Types.TIME:
    +                        stmt.setTime(parameterIndex, new Time(Long.parseLong(parameterValue)));
    +                        break;
    +                    case Types.TIMESTAMP:
    +                        stmt.setTimestamp(parameterIndex, new Timestamp(Long.parseLong(parameterValue)));
    +                        break;
    +                    case Types.CHAR:
    +                    case Types.VARCHAR:
    +                    case Types.LONGNVARCHAR:
    +                    case Types.LONGVARCHAR:
    +                        stmt.setString(parameterIndex, parameterValue);
    +                        break;
    +                    default:
    +                        stmt.setObject(parameterIndex, parameterValue, jdbcType);
    +                        break;
    +                }
    --- End diff --
    
    As much as I don't like the above, we know now that setObject does not always do what it was intended to do so we stuck with it. . . Argh. . . just venting ;)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60474620
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    --- End diff --
    
    This is another copy-paste from DBCPConnectionPool, I imagine it's so the value can be put immediately into the DBCP DataSource. I agree it's unnecessarily awkward, will make the changes you suggest. This would make it inconsistent with the DBCPConnectionPool, but I think that should be ok


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 closed the pull request at:

    https://github.com/apache/nifi/pull/372


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60471765
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    +            .required(true)
    +            .addValidator(StandardValidators.INTEGER_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    private static List<PropertyDescriptor> properties;
    +
    +    private volatile BasicDataSource dataSource;
    +
    +    @Override
    +    protected void init(ControllerServiceInitializationContext config) throws InitializationException {
    +
    +        List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(DATABASE_URL);
    +        props.add(HIVE_CONFIGURATION_RESOURCES);
    +        props.add(DB_USER);
    +        props.add(DB_PASSWORD);
    +        props.add(MAX_WAIT_TIME);
    +        props.add(MAX_TOTAL_CONNECTIONS);
    +        properties = Collections.unmodifiableList(props);
    +    }
    +
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return properties;
    +    }
    +
    +    /**
    +     * 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 drv = HiveDriver.class.getName();
    +        final String user = context.getProperty(DB_USER).getValue();
    +        final String passw = context.getProperty(DB_PASSWORD).getValue();
    +        final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).asInteger();
    +
    +        dataSource = new BasicDataSource();
    +        dataSource.setDriverClassName(drv);
    +
    +        final String dburl = context.getProperty(DATABASE_URL).getValue();
    +
    +        dataSource.setMaxWait(maxWaitMillis);
    +        dataSource.setMaxActive(maxTotal);
    +
    +        dataSource.setUrl(dburl);
    +        dataSource.setUsername(user);
    +        dataSource.setPassword(passw);
    +    }
    +
    +    /**
    +     * Shutdown pool, close all open connections.
    +     */
    +    @OnDisabled
    +    public void shutdown() {
    +        try {
    +            dataSource.close();
    +        } catch (final SQLException e) {
    +            throw new ProcessException(e);
    --- End diff --
    
    Not sure why/if this needs to be done, another copy-paste from DBCPConnectionPool


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60468215
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java ---
    @@ -0,0 +1,320 @@
    +/*
    + * 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.hive;
    +
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.SeeAlso;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.dbcp.hive.HiveDBCPService;
    +import org.apache.nifi.flowfile.FlowFile;
    +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.InputStreamCallback;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.stream.io.StreamUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.math.BigDecimal;
    +import java.nio.charset.Charset;
    +import java.sql.Connection;
    +import java.sql.Date;
    +import java.sql.PreparedStatement;
    +import java.sql.SQLException;
    +import java.sql.SQLNonTransientException;
    +import java.sql.Time;
    +import java.sql.Timestamp;
    +import java.sql.Types;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SeeAlso(ExecuteHiveQL.class)
    +@InputRequirement(Requirement.INPUT_REQUIRED)
    +@Tags({"sql", "hive", "put", "database", "update", "insert"})
    +@CapabilityDescription("Executes a HiveQL DDL/DML command (UPDATE, INSERT, e.g.). The content of an incoming FlowFile is expected to be the HiveQL command "
    +        + "to execute. The HiveQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
    +        + "with the naming convention hiveql.args.N.type and hiveql.args.N.value, where N is a positive integer. The hiveql.args.N.type is expected to be "
    +        + "a number indicating the JDBC Type. The content of the FlowFile is expected to be in UTF-8 format.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "hiveql.args.N.type", description = "Incoming FlowFiles are expected to be parameterized HiveQL statements. The type of each Parameter is specified as an integer "
    +                + "that represents the JDBC Type of the parameter."),
    +        @ReadsAttribute(attribute = "hiveql.args.N.value", description = "Incoming FlowFiles are expected to be parameterized HiveQL statements. The value of the Parameters are specified as "
    +                + "hiveql.args.1.value, hiveql.args.2.value, hiveql.args.3.value, and so on. The type of the hiveql.args.1.value Parameter is specified by the hiveql.args.1.type attribute.")
    +})
    +public class PutHiveQL extends AbstractHiveQLProcessor {
    +
    +    public static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The preferred number of FlowFiles to put to the database in a single transaction")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    public static final PropertyDescriptor CHARSET = new PropertyDescriptor.Builder()
    +            .name("Character Set")
    +            .description("Specifies the character set of the record data.")
    +            .required(true)
    +            .defaultValue("UTF-8")
    +            .addValidator(StandardValidators.CHARACTER_SET_VALIDATOR)
    +            .build();
    +
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("A FlowFile is routed to this relationship after the database is successfully updated")
    +            .build();
    +    public static final Relationship REL_RETRY = new Relationship.Builder()
    +            .name("retry")
    +            .description("A FlowFile is routed to this relationship if the database cannot be updated but attempting the operation again may succeed")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("A FlowFile is routed to this relationship if the database cannot be updated and retrying the operation will also fail, "
    +                    + "such as an invalid query or an integrity constraint violation")
    +            .build();
    +
    +    private static final Pattern HIVEQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("hiveql\\.args\\.(\\d+)\\.type");
    +    private static final Pattern NUMBER_PATTERN = Pattern.compile("-?\\d+");
    +
    +    private final static List<PropertyDescriptor> propertyDescriptors;
    +    private final static Set<Relationship> relationships;
    +
    +    /*
    +     * Will ensure that the list of property descriptors is build only once.
    +     * Will also create a Set of relationships
    +     */
    +    static {
    +        List<PropertyDescriptor> _propertyDescriptors = new ArrayList<>();
    +        _propertyDescriptors.add(HIVE_DBCP_SERVICE);
    +        _propertyDescriptors.add(BATCH_SIZE);
    +        _propertyDescriptors.add(CHARSET);
    +        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
    +
    +        Set<Relationship> _relationships = new HashSet<>();
    +        _relationships.add(REL_SUCCESS);
    +        _relationships.add(REL_FAILURE);
    +        _relationships.add(REL_RETRY);
    +        relationships = Collections.unmodifiableSet(_relationships);
    +    }
    --- End diff --
    
    This looks familiar. Consider making it cocnsistent across


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60474410
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    --- End diff --
    
    Though I guess we are setting a Default Value here, so if you remove the value, it will just reset to 8. So I think I understand the reason for allowing a negative value now. We should probably consider allowing an optional property to have its value removed, even if there is a default value.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60467110
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    +            .required(true)
    +            .addValidator(StandardValidators.INTEGER_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    private static List<PropertyDescriptor> properties;
    +
    +    private volatile BasicDataSource dataSource;
    +
    +    @Override
    +    protected void init(ControllerServiceInitializationContext config) throws InitializationException {
    +
    +        List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(DATABASE_URL);
    +        props.add(HIVE_CONFIGURATION_RESOURCES);
    +        props.add(DB_USER);
    +        props.add(DB_PASSWORD);
    +        props.add(MAX_WAIT_TIME);
    +        props.add(MAX_TOTAL_CONNECTIONS);
    +        properties = Collections.unmodifiableList(props);
    +    }
    --- End diff --
    
    Given what we know and that is this method will get executed several times before the processor will actually shows up on the canvas I'd suggest using a bit different approach. For example: https://github.com/apache/nifi/blob/master/nifi-nar-bundles/nifi-amqp-bundle/nifi-amqp-processors/src/main/java/org/apache/nifi/amqp/processors/PublishAMQP.java#L102


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60466456
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    --- End diff --
    
    Matt, one thing that has been actually discussed in the couple of recent PRs was to start embracing ```.displayName(string)```. The main reason is that we are currently coupling what user's see to the actual field names. The problem with this approach is obvious. If at some point down the road we realize that we should have named something differently we can't do that since all the templates that are using it will break.
    
    With ```displayName``` we would essentially decouple what user sees from the the actual property name (even if initially they are the same). This way we are free to modify it in the future every way we want to make it more user friendly without breaking backwards compatibility.
    
    Obviously there is no precedence for this other then me and @alopresto joining forces in advocating it. So your choice, but I figured I'll mention. ;)


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60467669
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    +            .required(true)
    +            .addValidator(StandardValidators.INTEGER_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    private static List<PropertyDescriptor> properties;
    +
    +    private volatile BasicDataSource dataSource;
    +
    +    @Override
    +    protected void init(ControllerServiceInitializationContext config) throws InitializationException {
    +
    +        List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(DATABASE_URL);
    +        props.add(HIVE_CONFIGURATION_RESOURCES);
    +        props.add(DB_USER);
    +        props.add(DB_PASSWORD);
    +        props.add(MAX_WAIT_TIME);
    +        props.add(MAX_TOTAL_CONNECTIONS);
    +        properties = Collections.unmodifiableList(props);
    +    }
    +
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return properties;
    +    }
    +
    +    /**
    +     * 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 drv = HiveDriver.class.getName();
    +        final String user = context.getProperty(DB_USER).getValue();
    +        final String passw = context.getProperty(DB_PASSWORD).getValue();
    +        final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).asInteger();
    +
    +        dataSource = new BasicDataSource();
    +        dataSource.setDriverClassName(drv);
    +
    +        final String dburl = context.getProperty(DATABASE_URL).getValue();
    +
    +        dataSource.setMaxWait(maxWaitMillis);
    +        dataSource.setMaxActive(maxTotal);
    +
    +        dataSource.setUrl(dburl);
    +        dataSource.setUsername(user);
    +        dataSource.setPassword(passw);
    +    }
    +
    +    /**
    +     * Shutdown pool, close all open connections.
    +     */
    +    @OnDisabled
    +    public void shutdown() {
    +        try {
    +            dataSource.close();
    +        } catch (final SQLException e) {
    +            throw new ProcessException(e);
    --- End diff --
    
    Do we really want to do that? What's the recovery from that? I think just logging it with WARN would be enough. It seems like a common pattern


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by markap14 <gi...@git.apache.org>.
Github user markap14 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60474156
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    --- End diff --
    
    This seems odd to me - why is this required with a negative value indicating no limit? Why not just make it optional and use a POSITIVE_INTEGER_VALIDATOR? Then, if you want no limit, just don't enter a value.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60467535
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    +            .required(true)
    +            .addValidator(StandardValidators.INTEGER_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    private static List<PropertyDescriptor> properties;
    +
    +    private volatile BasicDataSource dataSource;
    +
    +    @Override
    +    protected void init(ControllerServiceInitializationContext config) throws InitializationException {
    +
    +        List<PropertyDescriptor> props = new ArrayList<>();
    +        props.add(DATABASE_URL);
    +        props.add(HIVE_CONFIGURATION_RESOURCES);
    +        props.add(DB_USER);
    +        props.add(DB_PASSWORD);
    +        props.add(MAX_WAIT_TIME);
    +        props.add(MAX_TOTAL_CONNECTIONS);
    +        properties = Collections.unmodifiableList(props);
    +    }
    +
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return properties;
    +    }
    +
    +    /**
    +     * 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 drv = HiveDriver.class.getName();
    +        final String user = context.getProperty(DB_USER).getValue();
    +        final String passw = context.getProperty(DB_PASSWORD).getValue();
    +        final Long maxWaitMillis = context.getProperty(MAX_WAIT_TIME).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Integer maxTotal = context.getProperty(MAX_TOTAL_CONNECTIONS).asInteger();
    +
    +        dataSource = new BasicDataSource();
    +        dataSource.setDriverClassName(drv);
    +
    +        final String dburl = context.getProperty(DATABASE_URL).getValue();
    +
    +        dataSource.setMaxWait(maxWaitMillis);
    +        dataSource.setMaxActive(maxTotal);
    +
    +        dataSource.setUrl(dburl);
    +        dataSource.setUsername(user);
    +        dataSource.setPassword(passw);
    +    }
    --- End diff --
    
    Hmm, since it appears you are using DBCP, shouldn't you use the DBCP ControllerService?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60467301
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    --- End diff --
    
    Just curious, why 8? I mean as a default I still think I'd chose 1 (safest option) and have user bump it up as needed


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60475629
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    --- End diff --
    
    Personally I'd use 1 as a default and use 0 (a very common and well known convention to depict "no limit").


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by mattyb149 <gi...@git.apache.org>.
Github user mattyb149 commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60471604
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/dbcp/hive/HiveConnectionPool.java ---
    @@ -0,0 +1,184 @@
    +/*
    + * 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.dbcp.hive;
    +
    +import org.apache.commons.dbcp.BasicDataSource;
    +import org.apache.hive.jdbc.HiveDriver;
    +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.controller.AbstractControllerService;
    +import org.apache.nifi.controller.ConfigurationContext;
    +import org.apache.nifi.controller.ControllerServiceInitializationContext;
    +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.SQLException;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +
    +/**
    + * Implementation for Database Connection Pooling Service used for Apache Hive connections. Apache DBCP is used for connection pooling functionality.
    + */
    +@Tags({"hive", "dbcp", "jdbc", "database", "connection", "pooling", "store"})
    +@CapabilityDescription("Provides Database Connection Pooling Service for Apache Hive. Connections can be asked from pool and returned after usage.")
    +public class HiveConnectionPool extends AbstractControllerService implements HiveDBCPService {
    +
    +    public static final PropertyDescriptor DATABASE_URL = new PropertyDescriptor.Builder()
    +            .name("Database Connection URL")
    +            .description("A database connection URL used to connect to a database. May contain database system name, host, port, database name and some parameters."
    +                    + " The exact syntax of a database connection URL is specified by the Hive documentation. For example, the server principal is often included "
    +                    + "as a connection parameter when connecting to a secure Hive server.")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .required(true)
    +            .build();
    +
    +    public static final PropertyDescriptor HIVE_CONFIGURATION_RESOURCES = new PropertyDescriptor.Builder().name("Hive Configuration Resources")
    +            .description("A file or comma separated list of files which contains the Hive configuration (hive-site.xml, e.g.). Without this, Hadoop "
    +                    + "will search the classpath for a 'hive-site.xml' file or will revert to a default configuration.")
    +            .required(false).addValidator(StandardValidators.createMultipleFilesExistValidator()).build();
    +
    +    public static final PropertyDescriptor DB_USER = new PropertyDescriptor.Builder()
    +            .name("Database User")
    +            .description("Database user name")
    +            .defaultValue(null)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor DB_PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("The password for the database user")
    +            .defaultValue(null)
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_WAIT_TIME = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time that the pool will wait (when there are no available connections) "
    +                    + " for a connection to be returned before failing, or -1 to wait indefinitely. ")
    +            .defaultValue("500 millis")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .sensitive(false)
    +            .build();
    +
    +    public static final PropertyDescriptor MAX_TOTAL_CONNECTIONS = new PropertyDescriptor.Builder()
    +            .name("Max Total Connections")
    +            .description("The maximum number of active connections that can be allocated from this pool at the same time, "
    +                    + " or negative for no limit.")
    +            .defaultValue("8")
    --- End diff --
    
    Copied this from DBCPConnectionPool, had no reason to change it


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] nifi pull request: NIFI-981 [REVIEW-ONLY]: Initial addition of Hiv...

Posted by olegz <gi...@git.apache.org>.
Github user olegz commented on a diff in the pull request:

    https://github.com/apache/nifi/pull/372#discussion_r60467879
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -0,0 +1,172 @@
    +/*
    + * 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.hive;
    +
    +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.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.dbcp.hive.HiveDBCPService;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.logging.ProcessorLog;
    +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.util.LongHolder;
    +import org.apache.nifi.util.StopWatch;
    +import org.apache.nifi.util.hive.HiveJdbcCommon;
    +
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.sql.Connection;
    +import java.sql.ResultSet;
    +import java.sql.SQLException;
    +import java.sql.Statement;
    +import java.util.ArrayList;
    +import java.util.Collections;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Set;
    +import java.util.concurrent.TimeUnit;
    +
    +@EventDriven
    +@InputRequirement(Requirement.INPUT_ALLOWED)
    +@Tags({"hive", "sql", "select", "jdbc", "query", "database"})
    +@CapabilityDescription("Execute provided HiveQL SELECT query against a Hive database connection. 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 'executehiveql.row.count' indicates how many rows were selected.")
    +public class ExecuteHiveQL extends AbstractHiveQLProcessor {
    +
    +    public static final String RESULT_ROW_COUNT = "executehiveql.row.count";
    +
    +    // Relationships
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Successfully created FlowFile from HiveQL query result set.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("HiveQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
    +            .build();
    +    private final Set<Relationship> relationships;
    +
    +    public static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
    +            .name("HiveQL select query")
    +            .description("HiveQL select query")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    private final List<PropertyDescriptor> propDescriptors;
    +
    +    public ExecuteHiveQL() {
    +        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(HIVE_DBCP_SERVICE);
    +        pds.add(HIVEQL_SELECT_QUERY);
    +        propDescriptors = Collections.unmodifiableList(pds);
    +    }
    --- End diff --
    
    This one will be executed multiple times as well. Lot of unnecessary work for GC for no reason.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---