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