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/02/18 16:21:13 UTC

[GitHub] nifi pull request: [REVIEW ONLY] NIFI-901: Add QueryCassandra proc...

GitHub user mattyb149 opened a pull request:

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

    [REVIEW ONLY] NIFI-901: Add QueryCassandra processor

    I'd like to get some eyes on the AbstractCassandraProcessor and QueryCassandra classes especially, so I can make any changes before adding PutCassandra and avoid duplicate effort.

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

    $ git pull https://github.com/mattyb149/nifi NIFI-901

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

    https://github.com/apache/nifi/pull/237.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 #237
    
----
commit c209eb8839892b2b07331a3c43484a8adaa0c52b
Author: Matt Burgess <ma...@gmail.com>
Date:   2016-02-18T15:18:18Z

    NIFI-901: Add QueryCassandra processor

----


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56216128
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,463 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    public static final int DEFAULT_CASSANDRA_PORT = 9042;
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    +        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
    +            final List<String> esList = Arrays.asList(input.split(","));
    +            for (String hostnamePort : esList) {
    +                String[] addresses = hostnamePort.split(":");
    +                // Protect against invalid input like http://127.0.0.1:9042 (URL scheme should not be there)
    +                if (addresses.length != 2) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                            "Each entry must be in hostname:port form (no scheme such as http://, and port must be specified)")
    --- End diff --
    
    Ahha. Agreed!


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56236638
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,463 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    public static final int DEFAULT_CASSANDRA_PORT = 9042;
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    +        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
    +            final List<String> esList = Arrays.asList(input.split(","));
    +            for (String hostnamePort : esList) {
    +                String[] addresses = hostnamePort.split(":");
    +                // Protect against invalid input like http://127.0.0.1:9042 (URL scheme should not be there)
    +                if (addresses.length != 2) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                            "Each entry must be in hostname:port form (no scheme such as http://, and port must be specified)")
    +                            .valid(false).build();
    +                }
    +                // Validate the port
    +                String port = addresses[1].trim();
    +                ValidationResult portValidatorResult = StandardValidators.PORT_VALIDATOR.validate(subject, port, context);
    +                if (!portValidatorResult.isValid()) {
    +                    return portValidatorResult;
    +                }
    +
    +            }
    +            return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                    "Valid cluster definition").valid(true).build();
    +        }
    +    };
    +
    +    // Common descriptors
    +    public static final PropertyDescriptor CONTACT_POINTS = new PropertyDescriptor.Builder()
    +            .name("Cassandra Contact Points")
    +            .description("Contact points are addresses of Cassandra nodes. The list of contact points should be "
    +                    + "comma-separated and in hostname:port format. Example node1:port,node2:port,...."
    +                    + " The default client port for Cassandra is 9042, but the port(s) must be explicitly specified.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(HOSTNAME_PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor KEYSPACE = new PropertyDescriptor.Builder()
    +            .name("Keyspace")
    +            .description("The Cassandra Keyspace to connect to. If no keyspace is specified, the query will need to "
    +                    + "include the keyspace name before any table reference.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                    + "connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
    +            .name("Client Auth")
    +            .description("Client authentication policy when connecting to secure (TLS/SSL) cluster. "
    +                    + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
    +                    + "has been defined and enabled.")
    +            .required(false)
    +            .allowableValues(SSLContextService.ClientAuth.values())
    +            .defaultValue("REQUIRED")
    +            .build();
    +
    +    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to access the Cassandra cluster")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to access the Cassandra cluster")
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSISTENCY_LEVEL = new PropertyDescriptor.Builder()
    +            .name("Consistency Level")
    +            .description("The strategy for how many replicas must respond before results are returned.")
    +            .required(true)
    +            .allowableValues(ConsistencyLevel.values())
    +            .defaultValue("ONE")
    +            .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();
    +
    +    // Relationships
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Successfully created FlowFile from SQL query result set.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
    --- End diff --
    
    Should probably change the success description to be more ambiguous (since it's being used for Put and Query). For PutCassandraQL, the FlowFile isn't getting "created" it is just getting transferred after the CQL gets run. Also flowfiles will not always be penalized when they are routed to failure. Also for failure, is it right to say "query execution failed" for PutCassandraQL? Finally for retry, is it correct to say that a source processor is "routing" a newly created flowfile?
    
    It can be difficult to properly word relationships to be correct for multiple processors, may think about having them in the individual 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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56191239
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/PutCassandraQLTest.java ---
    @@ -0,0 +1,189 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.Configuration;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.Statement;
    +import com.datastax.driver.core.exceptions.InvalidQueryException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.UnavailableException;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.util.TestRunner;
    +import org.apache.nifi.util.TestRunners;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import static org.junit.Assert.fail;
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.anyLong;
    +import static org.mockito.Matchers.anyString;
    +import static org.mockito.Mockito.doReturn;
    +import static org.mockito.Mockito.doThrow;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +/**
    + * Unit tests for the PutCassandraQL processor
    + */
    +public class PutCassandraQLTest {
    +
    +    private TestRunner testRunner;
    +    private MockPutCassandraQL processor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        processor = new MockPutCassandraQL();
    +        testRunner = TestRunners.newTestRunner(processor);
    +    }
    +
    +    @Test
    +    public void testProcessor() {
    +        testRunner.setProperty(AbstractCassandraProcessor.CONTACT_POINTS, "localhost:9042");
    +        testRunner.assertValid();
    +        testRunner.setProperty(AbstractCassandraProcessor.PASSWORD, "password");
    +        testRunner.assertNotValid();
    +        testRunner.setProperty(AbstractCassandraProcessor.USERNAME, "username");
    +        testRunner.setProperty(AbstractCassandraProcessor.CONSISTENCY_LEVEL, "ONE");
    +        testRunner.assertValid();
    +
    +        testRunner.enqueue("INSERT INTO users (user_id, first_name, last_name, properties) VALUES ?, ?, ?, ?",
    +                new HashMap<String, String>() {
    +                    {
    +                        put("cql.args.1.type", "int");
    +                        put("cql.args.1.value", "1");
    +                        put("cql.args.2.type", "text");
    +                        put("cql.args.2.value", "Joe");
    +                        put("cql.args.3.type", "text");
    +                        // No value for arg 3 to test setNull
    +                        put("cql.args.4.type", "map<text,text>");
    +                        put("cql.args.4.value", "{'a':'Hello', 'b':'World'}");
    +                        put("cql.args.5.type", "list<boolean>");
    +                        put("cql.args.5.value", "[true,false,true]");
    +                        put("cql.args.6.type", "set<double>");
    +                        put("cql.args.6.value", "{1.0, 2.0}");
    +                        put("cql.args.7.type", "bigint");
    +                        put("cql.args.7.value", "20000000");
    +                        put("cql.args.8.type", "float");
    +                        put("cql.args.8.value", "1.0");
    +                        put("cql.args.9.type", "blob");
    +                        put("cql.args.9.value", "0xDEADBEEF");
    +
    +                    }
    +                });
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_SUCCESS, 1);
    +        testRunner.clearTransferState();
    +
    +        // Test exceptions
    --- End diff --
    
    Any thoughts on breaking each of these out into it's own test? It would allow for better readability and modularity when something does go wrong.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56237303
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/PutCassandraQLTest.java ---
    @@ -0,0 +1,219 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.Configuration;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSet;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.Statement;
    +import com.datastax.driver.core.exceptions.InvalidQueryException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.UnavailableException;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.util.TestRunner;
    +import org.apache.nifi.util.TestRunners;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.concurrent.TimeUnit;
    +import java.util.concurrent.TimeoutException;
    +
    +import static org.junit.Assert.fail;
    +import static org.mockito.Matchers.any;
    +import static org.mockito.Matchers.anyLong;
    +import static org.mockito.Matchers.anyString;
    +import static org.mockito.Mockito.doReturn;
    +import static org.mockito.Mockito.doThrow;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +/**
    + * Unit tests for the PutCassandraQL processor
    + */
    +public class PutCassandraQLTest {
    +
    +    private TestRunner testRunner;
    +    private MockPutCassandraQL processor;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        processor = new MockPutCassandraQL();
    +        testRunner = TestRunners.newTestRunner(processor);
    +    }
    +
    +    @Test
    +    public void testProcessorConfigValidity() {
    +        testRunner.setProperty(AbstractCassandraProcessor.CONTACT_POINTS, "localhost:9042");
    +        testRunner.assertValid();
    +        testRunner.setProperty(AbstractCassandraProcessor.PASSWORD, "password");
    +        testRunner.assertNotValid();
    +        testRunner.setProperty(AbstractCassandraProcessor.USERNAME, "username");
    +        testRunner.setProperty(AbstractCassandraProcessor.CONSISTENCY_LEVEL, "ONE");
    +        testRunner.assertValid();
    +    }
    +
    +    @Test
    +    public void testProcessorHappyPath() {
    +        setUpStandardTestConfig();
    +
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_SUCCESS, 1);
    +        testRunner.clearTransferState();
    +    }
    +
    +    @Test
    +    public void testProcessorInvalidQueryException() {
    +        setUpStandardTestConfig();
    +
    +        // Test exceptions
    +        processor.setExceptionToThrow(
    +                new InvalidQueryException(new InetSocketAddress("localhost", 9042), "invalid query"));
    +        testRunner.enqueue("UPDATE users SET cities = [ 'New York', 'Los Angeles' ] WHERE user_id = 'coast2coast';");
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_FAILURE, 1);
    +        testRunner.clearTransferState();
    +    }
    +
    +    @Test
    +    public void testProcessorUnavailableException() {
    +        setUpStandardTestConfig();
    +
    +        processor.setExceptionToThrow(
    +                new UnavailableException(new InetSocketAddress("localhost", 9042), ConsistencyLevel.ALL, 5, 2));
    +        testRunner.enqueue("UPDATE users SET cities = [ 'New York', 'Los Angeles' ] WHERE user_id = 'coast2coast';");
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_RETRY, 1);
    +    }
    +
    +    @Test
    +    public void testProcessorNoHostAvailableException() {
    +        setUpStandardTestConfig();
    +
    +        processor.setExceptionToThrow(new NoHostAvailableException(new HashMap<InetSocketAddress, Throwable>()));
    +        testRunner.enqueue("UPDATE users SET cities = [ 'New York', 'Los Angeles' ] WHERE user_id = 'coast2coast';");
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_RETRY, 1);
    +    }
    +
    +    @Test
    +    public void testProcessorProcessException() {
    +        setUpStandardTestConfig();
    +
    +        processor.setExceptionToThrow(new ProcessException());
    +        testRunner.enqueue("UPDATE users SET cities = [ 'New York', 'Los Angeles' ] WHERE user_id = 'coast2coast';");
    +        testRunner.run(1, true, true);
    +        testRunner.assertAllFlowFilesTransferred(PutCassandraQL.REL_FAILURE, 1);
    +    }
    +
    +    private void setUpStandardTestConfig() {
    --- End diff --
    
    In many tests "setUpStandardTestConfig" is used (enqueuing a flowfile), then another flowfile is enqueued and then testRunner is only run once. The second FlowFile is not 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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56235566
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,463 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    public static final int DEFAULT_CASSANDRA_PORT = 9042;
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    +        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
    +            final List<String> esList = Arrays.asList(input.split(","));
    +            for (String hostnamePort : esList) {
    +                String[] addresses = hostnamePort.split(":");
    +                // Protect against invalid input like http://127.0.0.1:9042 (URL scheme should not be there)
    +                if (addresses.length != 2) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                            "Each entry must be in hostname:port form (no scheme such as http://, and port must be specified)")
    +                            .valid(false).build();
    +                }
    +                // Validate the port
    +                String port = addresses[1].trim();
    +                ValidationResult portValidatorResult = StandardValidators.PORT_VALIDATOR.validate(subject, port, context);
    +                if (!portValidatorResult.isValid()) {
    +                    return portValidatorResult;
    +                }
    +
    +            }
    +            return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                    "Valid cluster definition").valid(true).build();
    +        }
    +    };
    +
    +    // Common descriptors
    +    public static final PropertyDescriptor CONTACT_POINTS = new PropertyDescriptor.Builder()
    +            .name("Cassandra Contact Points")
    +            .description("Contact points are addresses of Cassandra nodes. The list of contact points should be "
    +                    + "comma-separated and in hostname:port format. Example node1:port,node2:port,...."
    +                    + " The default client port for Cassandra is 9042, but the port(s) must be explicitly specified.")
    +            .required(true)
    +            .expressionLanguageSupported(false)
    +            .addValidator(HOSTNAME_PORT_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor KEYSPACE = new PropertyDescriptor.Builder()
    +            .name("Keyspace")
    +            .description("The Cassandra Keyspace to connect to. If no keyspace is specified, the query will need to "
    +                    + "include the keyspace name before any table reference.")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PROP_SSL_CONTEXT_SERVICE = new PropertyDescriptor.Builder()
    +            .name("SSL Context Service")
    +            .description("The SSL Context Service used to provide client certificate information for TLS/SSL "
    +                    + "connections.")
    +            .required(false)
    +            .identifiesControllerService(SSLContextService.class)
    +            .build();
    +
    +    public static final PropertyDescriptor CLIENT_AUTH = new PropertyDescriptor.Builder()
    +            .name("Client Auth")
    +            .description("Client authentication policy when connecting to secure (TLS/SSL) cluster. "
    +                    + "Possible values are REQUIRED, WANT, NONE. This property is only used when an SSL Context "
    +                    + "has been defined and enabled.")
    +            .required(false)
    +            .allowableValues(SSLContextService.ClientAuth.values())
    +            .defaultValue("REQUIRED")
    +            .build();
    +
    +    public static final PropertyDescriptor USERNAME = new PropertyDescriptor.Builder()
    +            .name("Username")
    +            .description("Username to access the Cassandra cluster")
    +            .required(false)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor PASSWORD = new PropertyDescriptor.Builder()
    +            .name("Password")
    +            .description("Password to access the Cassandra cluster")
    +            .required(false)
    +            .sensitive(true)
    +            .addValidator(StandardValidators.NON_EMPTY_VALIDATOR)
    +            .build();
    +
    +    public static final PropertyDescriptor CONSISTENCY_LEVEL = new PropertyDescriptor.Builder()
    +            .name("Consistency Level")
    +            .description("The strategy for how many replicas must respond before results are returned.")
    +            .required(true)
    +            .allowableValues(ConsistencyLevel.values())
    +            .defaultValue("ONE")
    +            .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();
    +
    +    // Relationships
    +    public static final Relationship REL_SUCCESS = new Relationship.Builder()
    +            .name("success")
    +            .description("Successfully created FlowFile from SQL query result set.")
    +            .build();
    +    public static final Relationship REL_FAILURE = new Relationship.Builder()
    +            .name("failure")
    +            .description("SQL query execution failed. Incoming FlowFile will be penalized and routed to this relationship")
    --- End diff --
    
    The descriptions for the relationships reference SQL. I don't believe that's correct.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56216735
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,383 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "cql.args.N.type",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The type of each "
    +                        + "parameter is specified as a lowercase string corresponding to the Cassandra data type (text, "
    +                        + "int, boolean, e.g.). In the case of collections, the primitive type(s) of the elements in the "
    +                        + "collection should be comma-delimited, follow the collection type, and be enclosed in angle brackets "
    +                        + "(< and >), for example set<text> or map<timestamp, int>."),
    +        @ReadsAttribute(attribute = "cql.args.N.value",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The value of the "
    +                        + "parameters are specified as cql.args.1.value, cql.args.2.value, cql.args.3.value, and so on. The "
    +                        + " type of the cql.args.1.value parameter is specified by the cql.args.1.type attribute.")
    +})
    +public class PutCassandraQL extends AbstractCassandraProcessor {
    +
    +    static final PropertyDescriptor STATEMENT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running CQL select query. Must be of format "
    +                    + "<duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported "
    +                    + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +
    +    private static final Pattern CQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("cql\\.args\\.(\\d+)\\.type");
    +
    +    // Matches on top-level type (primitive types like text,int) and also for collections (like list<boolean> and map<float,double>)
    +    private static final Pattern CQL_TYPE_PATTERN = Pattern.compile("([^<]+)(<([^,>]+)(,([^,>]+))*>)?");
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(CONTACT_POINTS);
    +        descriptors.add(KEYSPACE);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(CLIENT_AUTH);
    +        descriptors.add(USERNAME);
    +        descriptors.add(PASSWORD);
    +        descriptors.add(STATEMENT_TIMEOUT);
    +        descriptors.add(CONSISTENCY_LEVEL);
    +        descriptors.add(CHARSET);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        relationships.add(REL_RETRY);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        ProcessorLog log = getLogger();
    +        try {
    +            connectToCassandra(context);
    +        } catch (final NoHostAvailableException nhae) {
    +            log.error("No host in the Cassandra cluster can be contacted successfully to execute this statement", nhae);
    +            // Log up to 10 error messages. Otherwise if a 1000-node cluster was specified but there was no connectivity,
    +            // a thousand error messages would be logged. However we would like information from Cassandra itself, so
    +            // cap the error limit at 10, format the messages, and don't include the stack trace (it is displayed by the
    +            // logger message above).
    +            log.error(nhae.getCustomMessage(10, true, false));
    +            throw new ProcessException(nhae);
    +        } catch (final AuthenticationException ae) {
    +            log.error("Invalid username/password combination", ae);
    +            throw new ProcessException(ae);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        ProcessorLog logger = getLogger();
    +        FlowFile flowFile = session.get();
    +        if (flowFile == null) {
    +            return;
    +        }
    +
    +        final long startNanos = System.nanoTime();
    +        final long statementTimeout = context.getProperty(STATEMENT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
    +
    +        // The documentation for the driver recommends the session remain open the entire time the processor is running
    +        // and states that it is thread-safe. This is why connectionSession is not in a try-with-resources.
    +        final Session connectionSession = cassandraSession.get();
    +
    +        String cql = getCQL(session, flowFile, charset);
    +        try {
    +            PreparedStatement statement = connectionSession.prepare(cql);
    +            BoundStatement boundStatement = statement.bind();
    +
    +            Map<String, String> attributes = flowFile.getAttributes();
    +            for (final Map.Entry<String, String> entry : attributes.entrySet()) {
    +                final String key = entry.getKey();
    +                final Matcher matcher = CQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
    +                if (matcher.matches()) {
    +                    final int parameterIndex = Integer.parseInt(matcher.group(1));
    +                    String paramType = entry.getValue();
    +                    if (StringUtils.isEmpty(paramType)) {
    +                        throw new ProcessException("Value of the " + key + " attribute is null or empty, it must contain a valid value");
    --- End diff --
    
    Hmm. . . what will happen with the FlowFile? Based on conversation I had with @markap14  a while back we should not be throwing ProcessExceptions from onTrigger(). At least without explicitly disposing of FlowFile (transfer, remove etc)


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56237562
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/test/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessorTest.java ---
    @@ -0,0 +1,284 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.Configuration;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.google.common.collect.Sets;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.ProcessSession;
    +import org.apache.nifi.processor.exception.ProcessException;
    +import org.apache.nifi.ssl.SSLContextService;
    +import org.apache.nifi.util.TestRunner;
    +import org.apache.nifi.util.TestRunners;
    +import org.junit.Before;
    +import org.junit.Test;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.Arrays;
    +import java.util.Calendar;
    +import java.util.Date;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +
    +import static org.junit.Assert.assertEquals;
    +import static org.junit.Assert.assertNotNull;
    +import static org.junit.Assert.assertNull;
    +import static org.junit.Assert.assertTrue;
    +import static org.mockito.Mockito.mock;
    +import static org.mockito.Mockito.when;
    +
    +
    +/**
    + * Unit tests for the AbstractCassandraProcessor class
    + */
    +public class AbstractCassandraProcessorTest {
    +
    +    MockAbstractCassandraProcessor processor;
    +    private TestRunner testRunner;
    +
    +    @Before
    +    public void setUp() throws Exception {
    +        processor = new MockAbstractCassandraProcessor();
    +        testRunner = TestRunners.newTestRunner(processor);
    +    }
    +
    +    @Test
    +    public void testCustomValidate() throws Exception {
    --- End diff --
    
    Since you already have a testCustomValidate, might as well add check for testing a port that isn't in the valid range.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56216003
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,463 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    public static final int DEFAULT_CASSANDRA_PORT = 9042;
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    +        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
    +            final List<String> esList = Arrays.asList(input.split(","));
    +            for (String hostnamePort : esList) {
    +                String[] addresses = hostnamePort.split(":");
    +                // Protect against invalid input like http://127.0.0.1:9042 (URL scheme should not be there)
    +                if (addresses.length != 2) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                            "Each entry must be in hostname:port form (no scheme such as http://, and port must be specified)")
    --- End diff --
    
    Sorry I meant to reply before. This is a list of hostname:port entries, I think it would be more confusing to have one property for a hostname list followed by an associated list of ports. That's why I added a single validator which now also validates the port, and that validator is applied to each entry in the 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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56187278
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,403 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type. The content of the FlowFile is expected to be in UTF-8 format.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "cql.args.N.type",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The type of each "
    +                        + "parameter is specified as a lowercase string corresponding to the Cassandra data type (text, "
    +                        + "int, boolean, e.g.). In the case of collections, the primitive type(s) of the elements in the "
    +                        + "collection should be comma-delimited, follow the collection type, and be enclosed in angle brackets "
    +                        + "(< and >), for example set<text> or map<timestamp, int>."),
    +        @ReadsAttribute(attribute = "cql.args.N.value",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The value of the "
    +                        + "parameters are specified as cql.args.1.value, cql.args.2.value, cql.args.3.value, and so on. The "
    +                        + " type of the cql.args.1.value parameter is specified by the cql.args.1.type attribute.")
    +})
    +public class PutCassandraQL extends AbstractCassandraProcessor {
    +
    +    static final PropertyDescriptor STATEMENT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running CQL select query. Must be of format "
    +                    + "<duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported "
    +                    + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +    static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The preferred number of FlowFiles to put to Cassandra in a single transaction")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    private static final Pattern CQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("cql\\.args\\.(\\d+)\\.type");
    +
    +    // Matches on top-level type (primitive types like text,int) and also for
    --- End diff --
    
    I think this comment is missing the second half of the sentence


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56214671
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,463 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    public static final int DEFAULT_CASSANDRA_PORT = 9042;
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    +        public ValidationResult validate(final String subject, final String input, final ValidationContext context) {
    +            final List<String> esList = Arrays.asList(input.split(","));
    +            for (String hostnamePort : esList) {
    +                String[] addresses = hostnamePort.split(":");
    +                // Protect against invalid input like http://127.0.0.1:9042 (URL scheme should not be there)
    +                if (addresses.length != 2) {
    +                    return new ValidationResult.Builder().subject(subject).input(input).explanation(
    +                            "Each entry must be in hostname:port form (no scheme such as http://, and port must be specified)")
    --- End diff --
    
    I may have commented on this before. Wouldn't it be easier to simply have two fields host/port and avid having this validator all together?


---
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-901: Add QueryCassandra and PutCassandraQL...

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

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


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#issuecomment-197461709
  
    +1
    
    Did a contrib check build, verified comments were addressed and ran through a Put and Query test with Cassandra 2.2.5 with no username/password, with just username/password and with SSL+username/password.
    
    Good work!


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r54299220
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,441 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    --- End diff --
    
    Don't you think it would be easier to follow a common convention and simply have 'host', 'port' properties? 


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56276158
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,383 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "cql.args.N.type",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The type of each "
    +                        + "parameter is specified as a lowercase string corresponding to the Cassandra data type (text, "
    +                        + "int, boolean, e.g.). In the case of collections, the primitive type(s) of the elements in the "
    +                        + "collection should be comma-delimited, follow the collection type, and be enclosed in angle brackets "
    +                        + "(< and >), for example set<text> or map<timestamp, int>."),
    +        @ReadsAttribute(attribute = "cql.args.N.value",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The value of the "
    +                        + "parameters are specified as cql.args.1.value, cql.args.2.value, cql.args.3.value, and so on. The "
    +                        + " type of the cql.args.1.value parameter is specified by the cql.args.1.type attribute.")
    +})
    +public class PutCassandraQL extends AbstractCassandraProcessor {
    +
    +    static final PropertyDescriptor STATEMENT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running CQL select query. Must be of format "
    +                    + "<duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported "
    +                    + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +
    +    private static final Pattern CQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("cql\\.args\\.(\\d+)\\.type");
    +
    +    // Matches on top-level type (primitive types like text,int) and also for collections (like list<boolean> and map<float,double>)
    +    private static final Pattern CQL_TYPE_PATTERN = Pattern.compile("([^<]+)(<([^,>]+)(,([^,>]+))*>)?");
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(CONTACT_POINTS);
    +        descriptors.add(KEYSPACE);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(CLIENT_AUTH);
    +        descriptors.add(USERNAME);
    +        descriptors.add(PASSWORD);
    +        descriptors.add(STATEMENT_TIMEOUT);
    +        descriptors.add(CONSISTENCY_LEVEL);
    +        descriptors.add(CHARSET);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        relationships.add(REL_RETRY);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        ProcessorLog log = getLogger();
    +        try {
    +            connectToCassandra(context);
    +        } catch (final NoHostAvailableException nhae) {
    +            log.error("No host in the Cassandra cluster can be contacted successfully to execute this statement", nhae);
    +            // Log up to 10 error messages. Otherwise if a 1000-node cluster was specified but there was no connectivity,
    +            // a thousand error messages would be logged. However we would like information from Cassandra itself, so
    +            // cap the error limit at 10, format the messages, and don't include the stack trace (it is displayed by the
    +            // logger message above).
    +            log.error(nhae.getCustomMessage(10, true, false));
    +            throw new ProcessException(nhae);
    +        } catch (final AuthenticationException ae) {
    +            log.error("Invalid username/password combination", ae);
    +            throw new ProcessException(ae);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        ProcessorLog logger = getLogger();
    +        FlowFile flowFile = session.get();
    +        if (flowFile == null) {
    +            return;
    +        }
    +
    +        final long startNanos = System.nanoTime();
    +        final long statementTimeout = context.getProperty(STATEMENT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
    +
    +        // The documentation for the driver recommends the session remain open the entire time the processor is running
    +        // and states that it is thread-safe. This is why connectionSession is not in a try-with-resources.
    +        final Session connectionSession = cassandraSession.get();
    +
    +        String cql = getCQL(session, flowFile, charset);
    +        try {
    +            PreparedStatement statement = connectionSession.prepare(cql);
    +            BoundStatement boundStatement = statement.bind();
    +
    +            Map<String, String> attributes = flowFile.getAttributes();
    +            for (final Map.Entry<String, String> entry : attributes.entrySet()) {
    +                final String key = entry.getKey();
    +                final Matcher matcher = CQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
    +                if (matcher.matches()) {
    +                    final int parameterIndex = Integer.parseInt(matcher.group(1));
    +                    String paramType = entry.getValue();
    +                    if (StringUtils.isEmpty(paramType)) {
    +                        throw new ProcessException("Value of the " + key + " attribute is null or empty, it must contain a valid value");
    --- End diff --
    
    The ProcessException(s) are caught below and the flow file is transferred to failure.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r54302766
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/AbstractCassandraProcessor.java ---
    @@ -0,0 +1,441 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.Cluster;
    +import com.datastax.driver.core.ConsistencyLevel;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.Metadata;
    +import com.datastax.driver.core.Row;
    +import com.datastax.driver.core.SSLOptions;
    +import com.datastax.driver.core.Session;
    +import org.apache.avro.Schema;
    +import org.apache.avro.SchemaBuilder;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.nifi.authorization.exception.ProviderCreationException;
    +import org.apache.nifi.components.PropertyDescriptor;
    +import org.apache.nifi.components.PropertyValue;
    +import org.apache.nifi.components.ValidationContext;
    +import org.apache.nifi.components.ValidationResult;
    +import org.apache.nifi.components.Validator;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.apache.nifi.processor.AbstractProcessor;
    +import org.apache.nifi.processor.ProcessContext;
    +import org.apache.nifi.processor.Relationship;
    +import org.apache.nifi.processor.util.StandardValidators;
    +import org.apache.nifi.security.util.SslContextFactory;
    +import org.apache.nifi.ssl.SSLContextService;
    +
    +import javax.net.ssl.SSLContext;
    +import java.net.InetSocketAddress;
    +import java.util.ArrayList;
    +import java.util.Arrays;
    +import java.util.Collection;
    +import java.util.HashSet;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Set;
    +import java.util.concurrent.atomic.AtomicReference;
    +
    +/**
    + * AbstractCassandraProcessor is a base class for Cassandra processors and contains logic and variables common to most
    + * processors integrating with Apache Cassandra.
    + */
    +public abstract class AbstractCassandraProcessor extends AbstractProcessor {
    +
    +    private static final Validator HOSTNAME_PORT_VALIDATOR = new Validator() {
    +        @Override
    --- End diff --
    
    In Cassandra's case it's a list of host:port, I don't think we have a standard validator for such, even if we break them into host & port.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56176199
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,403 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type. The content of the FlowFile is expected to be in UTF-8 format.")
    --- End diff --
    
    Doh! Good catch, will update


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56215391
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,383 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "cql.args.N.type",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The type of each "
    +                        + "parameter is specified as a lowercase string corresponding to the Cassandra data type (text, "
    +                        + "int, boolean, e.g.). In the case of collections, the primitive type(s) of the elements in the "
    +                        + "collection should be comma-delimited, follow the collection type, and be enclosed in angle brackets "
    +                        + "(< and >), for example set<text> or map<timestamp, int>."),
    +        @ReadsAttribute(attribute = "cql.args.N.value",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The value of the "
    +                        + "parameters are specified as cql.args.1.value, cql.args.2.value, cql.args.3.value, and so on. The "
    +                        + " type of the cql.args.1.value parameter is specified by the cql.args.1.type attribute.")
    +})
    +public class PutCassandraQL extends AbstractCassandraProcessor {
    +
    +    static final PropertyDescriptor STATEMENT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running CQL select query. Must be of format "
    +                    + "<duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported "
    +                    + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +
    +    private static final Pattern CQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("cql\\.args\\.(\\d+)\\.type");
    +
    +    // Matches on top-level type (primitive types like text,int) and also for collections (like list<boolean> and map<float,double>)
    +    private static final Pattern CQL_TYPE_PATTERN = Pattern.compile("([^<]+)(<([^,>]+)(,([^,>]+))*>)?");
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    --- End diff --
    
    While a common pattern in many of our processors, given the fact that its a known issue where _init()_ method is invoked several times, I'd suggest moving _relationships_ and _descriptors_ initialization to a static initializer to avoid unnecessary GC. You can look at AMQP, new JMS as well as SNMP PR.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56175269
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,403 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type. The content of the FlowFile is expected to be in UTF-8 format.")
    --- End diff --
    
    "The content of the FlowFile is expected to be in UTF-8 format." After adding the charset property I don't believe this is the case.


---
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-901: Add QueryCassandra and PutCassandraQL...

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

    https://github.com/apache/nifi/pull/237#discussion_r56178725
  
    --- Diff: nifi-nar-bundles/nifi-cassandra-bundle/nifi-cassandra-processors/src/main/java/org/apache/nifi/processors/cassandra/PutCassandraQL.java ---
    @@ -0,0 +1,403 @@
    +/*
    + * 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.cassandra;
    +
    +import com.datastax.driver.core.BoundStatement;
    +import com.datastax.driver.core.DataType;
    +import com.datastax.driver.core.PreparedStatement;
    +import com.datastax.driver.core.ResultSetFuture;
    +import com.datastax.driver.core.Session;
    +import com.datastax.driver.core.exceptions.AuthenticationException;
    +import com.datastax.driver.core.exceptions.InvalidTypeException;
    +import com.datastax.driver.core.exceptions.NoHostAvailableException;
    +import com.datastax.driver.core.exceptions.QueryExecutionException;
    +import com.datastax.driver.core.exceptions.QueryValidationException;
    +import org.apache.nifi.annotation.behavior.EventDriven;
    +import org.apache.nifi.annotation.behavior.InputRequirement;
    +import org.apache.nifi.annotation.behavior.ReadsAttribute;
    +import org.apache.nifi.annotation.behavior.ReadsAttributes;
    +import org.apache.nifi.annotation.behavior.SupportsBatching;
    +import org.apache.nifi.annotation.documentation.CapabilityDescription;
    +import org.apache.nifi.annotation.documentation.Tags;
    +import org.apache.nifi.annotation.lifecycle.OnScheduled;
    +import org.apache.nifi.annotation.lifecycle.OnShutdown;
    +import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
    +import org.apache.nifi.components.PropertyDescriptor;
    +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.ProcessorInitializationContext;
    +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 org.apache.nifi.util.StringUtils;
    +
    +import java.io.IOException;
    +import java.io.InputStream;
    +import java.nio.ByteBuffer;
    +import java.nio.charset.Charset;
    +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.concurrent.TimeoutException;
    +import java.util.regex.Matcher;
    +import java.util.regex.Pattern;
    +
    +@SupportsBatching
    +@Tags({"cassandra", "cql", "put", "insert", "update", "set"})
    +@EventDriven
    +@InputRequirement(InputRequirement.Requirement.INPUT_REQUIRED)
    +@CapabilityDescription("Execute provided Cassandra Query Language (CQL) statement. The content of an incoming FlowFile "
    +        + "is expected to be the CQL command to execute. The CQL command may use the ? to escape parameters. In this "
    +        + "case, the parameters to use must exist as FlowFile attributes with the naming convention cql.args.N.type "
    +        + "and cql.args.N.value, where N is a positive integer. The cql.args.N.type is expected to be "
    +        + "a lowercase string indicating the Cassandra type. The content of the FlowFile is expected to be in UTF-8 format.")
    +@ReadsAttributes({
    +        @ReadsAttribute(attribute = "cql.args.N.type",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The type of each "
    +                        + "parameter is specified as a lowercase string corresponding to the Cassandra data type (text, "
    +                        + "int, boolean, e.g.). In the case of collections, the primitive type(s) of the elements in the "
    +                        + "collection should be comma-delimited, follow the collection type, and be enclosed in angle brackets "
    +                        + "(< and >), for example set<text> or map<timestamp, int>."),
    +        @ReadsAttribute(attribute = "cql.args.N.value",
    +                description = "Incoming FlowFiles are expected to be parameterized CQL statements. The value of the "
    +                        + "parameters are specified as cql.args.1.value, cql.args.2.value, cql.args.3.value, and so on. The "
    +                        + " type of the cql.args.1.value parameter is specified by the cql.args.1.type attribute.")
    +})
    +public class PutCassandraQL extends AbstractCassandraProcessor {
    +
    +    static final PropertyDescriptor STATEMENT_TIMEOUT = new PropertyDescriptor.Builder()
    +            .name("Max Wait Time")
    +            .description("The maximum amount of time allowed for a running CQL select query. Must be of format "
    +                    + "<duration> <TimeUnit> where <duration> is a non-negative integer and TimeUnit is a supported "
    +                    + "Time Unit, such as: nanos, millis, secs, mins, hrs, days. A value of zero means there is no limit. ")
    +            .defaultValue("0 seconds")
    +            .required(true)
    +            .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR)
    +            .build();
    +    static final PropertyDescriptor BATCH_SIZE = new PropertyDescriptor.Builder()
    +            .name("Batch Size")
    +            .description("The preferred number of FlowFiles to put to Cassandra in a single transaction")
    +            .required(true)
    +            .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR)
    +            .defaultValue("100")
    +            .build();
    +
    +    private static final Pattern CQL_TYPE_ATTRIBUTE_PATTERN = Pattern.compile("cql\\.args\\.(\\d+)\\.type");
    +
    +    // Matches on top-level type (primitive types like text,int) and also for
    +    private static final Pattern CQL_TYPE_PATTERN = Pattern.compile("([^<]+)(<([^,>]+)(,([^,>]+))*>)?");
    +
    +
    +    @Override
    +    protected void init(final ProcessorInitializationContext context) {
    +        final List<PropertyDescriptor> descriptors = new ArrayList<>();
    +        descriptors.add(CONTACT_POINTS);
    +        descriptors.add(KEYSPACE);
    +        descriptors.add(PROP_SSL_CONTEXT_SERVICE);
    +        descriptors.add(CLIENT_AUTH);
    +        descriptors.add(USERNAME);
    +        descriptors.add(PASSWORD);
    +        descriptors.add(STATEMENT_TIMEOUT);
    +        descriptors.add(BATCH_SIZE);
    +        descriptors.add(CONSISTENCY_LEVEL);
    +        descriptors.add(CHARSET);
    +        this.descriptors = Collections.unmodifiableList(descriptors);
    +
    +        final Set<Relationship> relationships = new HashSet<>();
    +        relationships.add(REL_SUCCESS);
    +        relationships.add(REL_FAILURE);
    +        relationships.add(REL_RETRY);
    +        this.relationships = Collections.unmodifiableSet(relationships);
    +    }
    +
    +    @Override
    +    protected List<PropertyDescriptor> getSupportedPropertyDescriptors() {
    +        return descriptors;
    +    }
    +
    +    @Override
    +    public Set<Relationship> getRelationships() {
    +        return relationships;
    +    }
    +
    +
    +    @OnScheduled
    +    public void onScheduled(final ProcessContext context) {
    +        ProcessorLog log = getLogger();
    +        try {
    +            connectToCassandra(context);
    +        } catch (final NoHostAvailableException nhae) {
    +            log.error("No host in the Cassandra cluster can be contacted successfully to execute this statement", nhae);
    +            // Log up to 10 error messages. Otherwise if a 1000-node cluster was specified but there was no connectivity,
    +            // a thousand error messages would be logged. However we would like information from Cassandra itself, so
    +            // cap the error limit at 10, format the messages, and don't include the stack trace (it is displayed by the
    +            // logger message above).
    +            log.error(nhae.getCustomMessage(10, true, false));
    +            throw new ProcessException(nhae);
    +        } catch (final AuthenticationException ae) {
    +            log.error("Invalid username/password combination", ae);
    +            throw new ProcessException(ae);
    +        }
    +    }
    +
    +    @Override
    +    public void onTrigger(final ProcessContext context, final ProcessSession session) throws ProcessException {
    +        final int batchSize = context.getProperty(BATCH_SIZE).asInteger();
    +        ProcessorLog logger = getLogger();
    +        final List<FlowFile> flowFiles = session.get(batchSize);
    +        if (flowFiles == null) {
    +            return;
    +        }
    +
    +        final long startNanos = System.nanoTime();
    +        final long statementTimeout = context.getProperty(STATEMENT_TIMEOUT).asTimePeriod(TimeUnit.MILLISECONDS);
    +        final Charset charset = Charset.forName(context.getProperty(CHARSET).getValue());
    +
    +        // The documentation for the driver recommends the session remain open the entire time the processor is running
    +        // and states that it is thread-safe. This is why connectionSession is not in a try-with-resources.
    +        final Session connectionSession = cassandraSession.get();
    +
    +        for (FlowFile flowFile : flowFiles) {
    +            String cql = getCQL(session, flowFile, charset);
    +            try {
    +                PreparedStatement statement = connectionSession.prepare(cql);
    +                BoundStatement boundStatement = statement.bind();
    +
    +                Map<String, String> attributes = flowFile.getAttributes();
    +                for (final Map.Entry<String, String> entry : attributes.entrySet()) {
    +                    final String key = entry.getKey();
    +                    final Matcher matcher = CQL_TYPE_ATTRIBUTE_PATTERN.matcher(key);
    +                    if (matcher.matches()) {
    +                        final int parameterIndex = Integer.parseInt(matcher.group(1));
    +                        String paramType = entry.getValue();
    +                        if (StringUtils.isEmpty(paramType)) {
    +                            throw new ProcessException("Value of the " + key + " attribute is null or empty, it must contain a valid value");
    +                        }
    +
    +                        paramType = paramType.trim();
    +                        final String valueAttrName = "cql.args." + parameterIndex + ".value";
    +                        final String parameterValue = attributes.get(valueAttrName);
    +
    +                        try {
    +                            setStatementObject(boundStatement, parameterIndex - 1, valueAttrName, parameterValue, paramType);
    +                        } catch (final InvalidTypeException | IllegalArgumentException e) {
    +                            throw new ProcessException("The value of the " + valueAttrName + " is '" + parameterValue
    +                                    + "', which cannot be converted into the necessary data type: " + paramType, e);
    +                        }
    +                    }
    +                }
    +
    +                try {
    +                    ResultSetFuture future = connectionSession.executeAsync(boundStatement);
    +                    if (statementTimeout > 0) {
    +                        future.getUninterruptibly(statementTimeout, TimeUnit.MILLISECONDS);
    +                    } else {
    +                        future.getUninterruptibly();
    +                    }
    +                    // Emit a Provenance SEND event
    +                    final long transmissionMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNanos);
    +
    +                    // This isn't a real URI but since Cassandra is distributed we just use the cluster name
    +                    String transitUri = "cassandra://" + connectionSession.getCluster().getMetadata().getClusterName();
    +                    session.getProvenanceReporter().send(flowFile, transitUri, transmissionMillis, true);
    +                    session.transfer(flowFile, REL_SUCCESS);
    +
    +                } catch (final TimeoutException e) {
    +                    throw new ProcessException(e);
    +                }
    +
    +
    +            } catch (final NoHostAvailableException nhae) {
    +                getLogger().error("No host in the Cassandra cluster can be contacted successfully to execute this statement", nhae);
    +                // Log up to 10 error messages. Otherwise if a 1000-node cluster was specified but there was no connectivity,
    +                // a thousand error messages would be logged. However we would like information from Cassandra itself, so
    +                // cap the error limit at 10, format the messages, and don't include the stack trace (it is displayed by the
    +                // logger message above).
    +                getLogger().error(nhae.getCustomMessage(10, true, false));
    +                flowFile = session.penalize(flowFile);
    +                session.transfer(flowFile, REL_RETRY);
    +
    +            } catch (final QueryExecutionException qee) {
    +                logger.error("Cannot execute the statement with the requested consistency level successfully", qee);
    +                // The incoming flow file should be penalized
    +                flowFile = session.penalize(flowFile);
    +                session.transfer(flowFile, REL_RETRY);
    +
    +            } catch (final QueryValidationException qve) {
    +                logger.error("The CQL statement {} is invalid due to syntax error, authorization issue, or another "
    +                                + "validation problem; routing {} to failure",
    +                        new Object[]{cql, flowFile}, qve);
    +                flowFile = session.penalize(flowFile);
    +                session.transfer(flowFile, REL_FAILURE);
    +
    +            } catch (final ProcessException e) {
    +                if (flowFile == null) {
    --- End diff --
    
    Since PutCassandraQL is not a source processor the FlowFile will never be null here.


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