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/27 20:40:40 UTC

[GitHub] nifi pull request: NIFI-981: Added ExecuteHiveQL and PutHiveQL pro...

GitHub user mattyb149 opened a pull request:

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

    NIFI-981: Added ExecuteHiveQL and PutHiveQL processors

    

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/384.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 #384
    
----
commit 70df8e0a095a98be5cea06a2ff56586584064f2d
Author: Matt Burgess <ma...@apache.org>
Date:   2016-04-27T18:39:11Z

    NIFI-981: Added ExecuteHiveQL and PutHiveQL processors

----


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61343688
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -0,0 +1,178 @@
    +/*
    + * 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();
    +
    +
    +    public static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
    +            .name("hive-query")
    --- End diff --
    
    One thing I noticed was when you drag on a brand new ExecuteHiveQL processor, the error message for the query will say "hive-query is invalid", I'm wondering if this is actually a bug in the validator where it is using the name instead of display name


---
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: Added ExecuteHiveQL and PutHiveQL pro...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the pull request:

    https://github.com/apache/nifi/pull/384#issuecomment-216246029
  
    Latest commits look good, I am a +1 and going to merge to 0.x and master


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61317547
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -0,0 +1,178 @@
    +/*
    + * 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();
    +
    +
    +    public static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
    +            .name("hive-query")
    +            .displayName("HiveQL select query")
    +            .description("HiveQL select query")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    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(HIVEQL_SELECT_QUERY);
    +        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
    +
    +        Set<Relationship> _relationships = new HashSet<>();
    +        _relationships.add(REL_SUCCESS);
    +        _relationships.add(REL_FAILURE);
    +        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 {
    +        FlowFile fileToProcess = null;
    +        if (context.hasIncomingConnection()) {
    +            fileToProcess = session.get();
    +
    +            // If we have no FlowFile, and all incoming connections are self-loops then we can continue on.
    +            // However, if we have no FlowFile and we have connections coming from other Processors, then
    +            // we know that we should run only if we have a FlowFile.
    +            if (fileToProcess == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final ProcessorLog logger = getLogger();
    +        final HiveDBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(HiveDBCPService.class);
    +        final String selectQuery = context.getProperty(HIVEQL_SELECT_QUERY).evaluateAttributeExpressions(fileToProcess).getValue();
    +        final StopWatch stopWatch = new StopWatch(true);
    +
    +        try (final Connection con = dbcpService.getConnection();
    +            final Statement st = con.createStatement()) {
    +            final LongHolder nrOfRows = new LongHolder(0L);
    +            if (fileToProcess == null) {
    +                fileToProcess = session.create();
    +            }
    +            fileToProcess = session.write(fileToProcess, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    try {
    +                        logger.debug("Executing query {}", new Object[]{selectQuery});
    +                        final ResultSet resultSet = st.executeQuery(selectQuery);
    +                        nrOfRows.set(HiveJdbcCommon.convertToAvroStream(resultSet, out));
    +                    } catch (final SQLException e) {
    +                        throw new ProcessException(e);
    +                    }
    +                }
    +            });
    +
    +            // set attribute how many rows were selected
    +            fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, nrOfRows.get().toString());
    +
    +            logger.info("{} contains {} Avro records; transferring to 'success'",
    +                    new Object[]{fileToProcess, nrOfRows.get()});
    +            session.getProvenanceReporter().modifyContent(fileToProcess, "Retrieved " + nrOfRows.get() + " rows",
    --- End diff --
    
    Good point, will make the change


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61316112
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -0,0 +1,178 @@
    +/*
    + * 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();
    +
    +
    +    public static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
    +            .name("hive-query")
    +            .displayName("HiveQL select query")
    +            .description("HiveQL select query")
    +            .required(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .expressionLanguageSupported(true)
    +            .build();
    +
    +    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(HIVEQL_SELECT_QUERY);
    +        propertyDescriptors = Collections.unmodifiableList(_propertyDescriptors);
    +
    +        Set<Relationship> _relationships = new HashSet<>();
    +        _relationships.add(REL_SUCCESS);
    +        _relationships.add(REL_FAILURE);
    +        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 {
    +        FlowFile fileToProcess = null;
    +        if (context.hasIncomingConnection()) {
    +            fileToProcess = session.get();
    +
    +            // If we have no FlowFile, and all incoming connections are self-loops then we can continue on.
    +            // However, if we have no FlowFile and we have connections coming from other Processors, then
    +            // we know that we should run only if we have a FlowFile.
    +            if (fileToProcess == null && context.hasNonLoopConnection()) {
    +                return;
    +            }
    +        }
    +
    +        final ProcessorLog logger = getLogger();
    +        final HiveDBCPService dbcpService = context.getProperty(HIVE_DBCP_SERVICE).asControllerService(HiveDBCPService.class);
    +        final String selectQuery = context.getProperty(HIVEQL_SELECT_QUERY).evaluateAttributeExpressions(fileToProcess).getValue();
    +        final StopWatch stopWatch = new StopWatch(true);
    +
    +        try (final Connection con = dbcpService.getConnection();
    +            final Statement st = con.createStatement()) {
    +            final LongHolder nrOfRows = new LongHolder(0L);
    +            if (fileToProcess == null) {
    +                fileToProcess = session.create();
    +            }
    +            fileToProcess = session.write(fileToProcess, new OutputStreamCallback() {
    +                @Override
    +                public void process(final OutputStream out) throws IOException {
    +                    try {
    +                        logger.debug("Executing query {}", new Object[]{selectQuery});
    +                        final ResultSet resultSet = st.executeQuery(selectQuery);
    +                        nrOfRows.set(HiveJdbcCommon.convertToAvroStream(resultSet, out));
    +                    } catch (final SQLException e) {
    +                        throw new ProcessException(e);
    +                    }
    +                }
    +            });
    +
    +            // set attribute how many rows were selected
    +            fileToProcess = session.putAttribute(fileToProcess, RESULT_ROW_COUNT, nrOfRows.get().toString());
    +
    +            logger.info("{} contains {} Avro records; transferring to 'success'",
    +                    new Object[]{fileToProcess, nrOfRows.get()});
    +            session.getProvenanceReporter().modifyContent(fileToProcess, "Retrieved " + nrOfRows.get() + " rows",
    --- End diff --
    
    What do you think about having different provenance events based on whether or not there was an incoming flow file? 
    
    If fileToProcess was null and we called session.create() we might want this to be a receive event with the uri of the Hive instance we pulled from, but if we had an incoming FlowFile and overwrote the content then we probably want modifyContent


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61317929
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/util/hive/HiveJdbcCommon.java ---
    @@ -0,0 +1,272 @@
    +/*
    + * 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.util.hive;
    +
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.avro.SchemaBuilder.FieldAssembler;
    +import org.apache.avro.file.DataFileWriter;
    +import org.apache.avro.generic.GenericData;
    +import org.apache.avro.generic.GenericDatumWriter;
    +import org.apache.avro.generic.GenericRecord;
    +import org.apache.avro.io.DatumWriter;
    +import org.apache.commons.lang3.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.OutputStream;
    +import java.math.BigDecimal;
    +import java.math.BigInteger;
    +import java.nio.ByteBuffer;
    +import java.sql.ResultSet;
    +import java.sql.ResultSetMetaData;
    +import java.sql.SQLException;
    +
    +import static java.sql.Types.ARRAY;
    +import static java.sql.Types.BIGINT;
    +import static java.sql.Types.BINARY;
    +import static java.sql.Types.BIT;
    +import static java.sql.Types.BLOB;
    +import static java.sql.Types.BOOLEAN;
    +import static java.sql.Types.CHAR;
    +import static java.sql.Types.CLOB;
    +import static java.sql.Types.DATE;
    +import static java.sql.Types.DECIMAL;
    +import static java.sql.Types.DOUBLE;
    +import static java.sql.Types.FLOAT;
    +import static java.sql.Types.INTEGER;
    +import static java.sql.Types.LONGNVARCHAR;
    +import static java.sql.Types.LONGVARBINARY;
    +import static java.sql.Types.LONGVARCHAR;
    +import static java.sql.Types.NCHAR;
    +import static java.sql.Types.NUMERIC;
    +import static java.sql.Types.NVARCHAR;
    +import static java.sql.Types.REAL;
    +import static java.sql.Types.ROWID;
    +import static java.sql.Types.SMALLINT;
    +import static java.sql.Types.TIME;
    +import static java.sql.Types.TIMESTAMP;
    +import static java.sql.Types.TINYINT;
    +import static java.sql.Types.VARBINARY;
    +import static java.sql.Types.VARCHAR;
    +
    +/**
    + * JDBC / HiveQL common functions.
    + */
    +public class HiveJdbcCommon {
    +
    +    public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream) throws SQLException, IOException {
    +        return convertToAvroStream(rs, outStream, null, null);
    +    }
    +
    +
    +    public static long convertToAvroStream(final ResultSet rs, final OutputStream outStream, String recordName, ResultSetRowCallback callback)
    +            throws SQLException, IOException {
    +        final Schema schema = createSchema(rs, recordName);
    +        final GenericRecord rec = new GenericData.Record(schema);
    +
    +        final DatumWriter<GenericRecord> datumWriter = new GenericDatumWriter<>(schema);
    +        try (final DataFileWriter<GenericRecord> dataFileWriter = new DataFileWriter<>(datumWriter)) {
    +            dataFileWriter.create(schema, outStream);
    +
    +            final ResultSetMetaData meta = rs.getMetaData();
    +            final int nrOfColumns = meta.getColumnCount();
    +            long nrOfRows = 0;
    +            while (rs.next()) {
    +                if (callback != null) {
    +                    callback.processRow(rs);
    +                }
    +                for (int i = 1; i <= nrOfColumns; i++) {
    +                    final int javaSqlType = meta.getColumnType(i);
    +                    final Object value = rs.getObject(i);
    +
    +                    if (value == null) {
    +                        rec.put(i - 1, null);
    +
    +                    } else if (javaSqlType == BINARY || javaSqlType == VARBINARY || javaSqlType == LONGVARBINARY || javaSqlType == ARRAY || javaSqlType == BLOB || javaSqlType == CLOB) {
    +                        // bytes requires little bit different handling
    +                        byte[] bytes = rs.getBytes(i);
    +                        ByteBuffer bb = ByteBuffer.wrap(bytes);
    +                        rec.put(i - 1, bb);
    +
    +                    } else if (value instanceof Byte) {
    +                        // tinyint(1) type is returned by JDBC driver as java.sql.Types.TINYINT
    +                        // But value is returned by JDBC as java.lang.Byte
    +                        // (at least H2 JDBC works this way)
    +                        // direct put to avro record results:
    +                        // org.apache.avro.AvroRuntimeException: Unknown datum type java.lang.Byte
    +                        rec.put(i - 1, ((Byte) value).intValue());
    +
    +                    } else if (value instanceof BigDecimal || value instanceof BigInteger) {
    +                        // Avro can't handle BigDecimal and BigInteger as numbers - it will throw an AvroRuntimeException such as: "Unknown datum type: java.math.BigDecimal: 38"
    +                        rec.put(i - 1, value.toString());
    +
    +                    } else if (value instanceof Number || value instanceof Boolean) {
    +                        rec.put(i - 1, value);
    +
    +                    } else {
    +                        // The different types that we support are numbers (int, long, double, float),
    +                        // as well as boolean values and Strings. Since Avro doesn't provide
    +                        // timestamp types, we want to convert those to Strings. So we will cast anything other
    +                        // than numbers or booleans to strings by using the toString() method.
    +                        rec.put(i - 1, value.toString());
    +                    }
    +                }
    +                dataFileWriter.append(rec);
    +                nrOfRows += 1;
    +            }
    +
    +            return nrOfRows;
    +        }
    +    }
    +
    +    public static Schema createSchema(final ResultSet rs) throws SQLException {
    +        return createSchema(rs, null);
    +    }
    +
    +    /**
    +     * Creates an Avro schema from a result set. If the table/record name is known a priori and provided, use that as a
    +     * fallback for the record name if it cannot be retrieved from the result set, and finally fall back to a default value.
    +     *
    +     * @param rs         The result set to convert to Avro
    +     * @param recordName The a priori record name to use if it cannot be determined from the result set.
    +     * @return A Schema object representing the result set converted to an Avro record
    +     * @throws SQLException if any error occurs during conversion
    +     */
    +    public static Schema createSchema(final ResultSet rs, String recordName) throws SQLException {
    +        final ResultSetMetaData meta = rs.getMetaData();
    +        final int nrOfColumns = meta.getColumnCount();
    +        String tableName = StringUtils.isEmpty(recordName) ? "NiFi_ExecuteSQL_Record" : recordName;
    --- End diff --
    
    Probably want this to be NiFi_HiveQL_Record, or something like that


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61486272
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -0,0 +1,178 @@
    +/*
    + * 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();
    +
    +
    +    public static final PropertyDescriptor HIVEQL_SELECT_QUERY = new PropertyDescriptor.Builder()
    +            .name("hive-query")
    --- End diff --
    
    the Validators use the "subject" passed in, and PropertyDescriptor.validate() passed in this.name instead of this.displayName. Think we should file a Jira? or discuss on the dev list?


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61832445
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java ---
    @@ -52,15 +54,27 @@
     @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."
    +@CapabilityDescription("Execute provided HiveQL SELECT query against a Hive database connection. Query result will be converted to Avro or CSV 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 {
    +@WritesAttributes({
    +        @WritesAttribute(attribute = "mime.type", description = "Sets the MIME type for the outgoing flowfile to application/avro-binary for Avro or text/csv for CSV."),
    +        @WritesAttribute(attribute = "filename", description = "Adds .avro or .csv to the filename attribute depending on which output format is selected."),
    +        @WritesAttribute(attribute = "executehiveql.row.count", description = "Indicates how many rows were selected/returned by the query.")
    --- End diff --
    
    Nit-picking here, but given the rename of the processor, do we want this to be selecthiveql.row.count?


---
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: Added ExecuteHiveQL and PutHiveQL pro...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the pull request:

    https://github.com/apache/nifi/pull/384#issuecomment-215498279
  
    Overall this looks great @mattyb149 ! I was able to get the processors working against kerberized and non-kerberized Hive. I think if we address the one or two minor comments then we should be good to go.


---
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: Added SelectHiveQL and PutHiveQL proc...

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

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


---
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: Added ExecuteHiveQL and PutHiveQL pro...

Posted by bbende <gi...@git.apache.org>.
Github user bbende commented on the pull request:

    https://github.com/apache/nifi/pull/384#issuecomment-216595481
  
    Latest changes look good, I am a +1 again :)


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61502555
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/ExecuteHiveQL.java ---
    @@ -152,8 +152,23 @@ public void process(final OutputStream out) throws IOException {
     
                 logger.info("{} contains {} Avro records; transferring to 'success'",
                         new Object[]{fileToProcess, nrOfRows.get()});
    -            session.getProvenanceReporter().modifyContent(fileToProcess, "Retrieved " + nrOfRows.get() + " rows",
    -                    stopWatch.getElapsed(TimeUnit.MILLISECONDS));
    +
    +            if (context.hasIncomingConnection()) {
    +                // If the flow file came from an incoming connection, issue a Modify Content provenance event
    +
    +                session.getProvenanceReporter().modifyContent(fileToProcess, "Retrieved " + nrOfRows.get() + " rows",
    +                        stopWatch.getElapsed(TimeUnit.MILLISECONDS));
    +            } else {
    +                // If we created a flow file from rows received from Hive, issue a Receive provenance event
    +                // Determine the database URL from the connection metadata
    +                String url = "jdbc:hive2://unknown-host";
    +                try {
    +                    url = con.getMetaData().getURL();
    --- End diff --
    
    In your testing did con.getMetaData().getUrl() return a value? 
    
    Testing on a VM I am always getting unknown-host in my provenance events. I'm wondering should we just take the value from the Database Connection URL property and use that as the URI here, instead of relying on the connection object.


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61832456
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/SelectHiveQL.java ---
    @@ -52,15 +54,27 @@
     @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."
    +@CapabilityDescription("Execute provided HiveQL SELECT query against a Hive database connection. Query result will be converted to Avro or CSV 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 {
    +@WritesAttributes({
    +        @WritesAttribute(attribute = "mime.type", description = "Sets the MIME type for the outgoing flowfile to application/avro-binary for Avro or text/csv for CSV."),
    +        @WritesAttribute(attribute = "filename", description = "Adds .avro or .csv to the filename attribute depending on which output format is selected."),
    +        @WritesAttribute(attribute = "executehiveql.row.count", description = "Indicates how many rows were selected/returned by the query.")
    +})
    +public class SelectHiveQL extends AbstractHiveQLProcessor {
     
         public static final String RESULT_ROW_COUNT = "executehiveql.row.count";
    --- End diff --
    
    same as above


---
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: Added ExecuteHiveQL and PutHiveQL pro...

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

    https://github.com/apache/nifi/pull/384#discussion_r61614773
  
    --- Diff: nifi-nar-bundles/nifi-hive-bundle/nifi-hive-processors/src/main/java/org/apache/nifi/processors/hive/PutHiveQL.java ---
    @@ -0,0 +1,322 @@
    +/*
    + * 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("hive-batch-size")
    +            .displayName("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("hive-charset")
    +            .displayName("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";
    --- End diff --
    
    We should change this to do the same thing as ExecuteSQL with the connection string from the connection pool right?


---
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.
---