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

Reply via email to