mirror of https://github.com/apache/nifi.git
NIFI-810: Created RequiresInput annotation and ensure that processors are invalid if connections do not agree
This commit is contained in:
parent
96764ed6a1
commit
034ee6de6b
|
@ -0,0 +1,51 @@
|
|||
package org.apache.nifi.annotation.behavior;
|
||||
|
||||
import java.lang.annotation.Documented;
|
||||
import java.lang.annotation.ElementType;
|
||||
import java.lang.annotation.Inherited;
|
||||
import java.lang.annotation.Retention;
|
||||
import java.lang.annotation.RetentionPolicy;
|
||||
import java.lang.annotation.Target;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Marker interface that a Processor can use to indicate whether it accepts, requires, or forbids
|
||||
* input from other Processors. This information is used by the framework in order to ensure that
|
||||
* a Processor is marked as invalid if it is missing necessary input or has input that will be ignored.
|
||||
* This information also is used by the NiFi UI in order to prevent users from making connections
|
||||
* to Processors that don't make sense.
|
||||
* </p>
|
||||
*/
|
||||
@Documented
|
||||
@Target({ElementType.TYPE})
|
||||
@Retention(RetentionPolicy.RUNTIME)
|
||||
@Inherited
|
||||
public @interface InputRequirement {
|
||||
Requirement value();
|
||||
|
||||
public static enum Requirement {
|
||||
/**
|
||||
* This value is used to indicate that the Processor requires input from other Processors
|
||||
* in order to run. As a result, the Processor will not be valid if it does not have any
|
||||
* incoming connections.
|
||||
*/
|
||||
INPUT_REQUIRED,
|
||||
|
||||
/**
|
||||
* This value is used to indicate that the Processor will consume data from an incoming
|
||||
* connection but does not require an incoming connection in order to perform its task.
|
||||
* If the {@link InputRequirement} annotation is not present, this is the default value
|
||||
* that is used.
|
||||
*/
|
||||
INPUT_ALLOWED,
|
||||
|
||||
/**
|
||||
* This value is used to indicate that the Processor is a "Source Processor" and does
|
||||
* not accept incoming connections. Because the Processor does not pull FlowFiles from
|
||||
* an incoming connection, it can be very confusing for users who create incoming connections
|
||||
* to the Processor. As a result, this value can be used in order to clarify that incoming
|
||||
* connections will not be used. This prevents the user from even creating such a connection.
|
||||
*/
|
||||
INPUT_FORBIDDEN;
|
||||
}
|
||||
}
|
|
@ -1633,6 +1633,17 @@ will handle your Processor:
|
|||
not there is any data on an input queue. This is useful, for example, if the Processor needs to be triggered to run
|
||||
periodically to time out a network connection.
|
||||
|
||||
- `InputRequirement`: By default, all Processors will allow users to create incoming connections for the Processor, but
|
||||
if the user does not create an incoming connection, the Processor is still valid and can be scheduled to run. For Processors
|
||||
that are expected to be used as a "Source Processor," though, this can be confusing to the user, and the user may attempt to
|
||||
send FlowFiles to that Processor, only for the FlowFiles to queue up without being processed. Conversely, if the Processor
|
||||
expects incoming FlowFiles but does not have an input queue, the Processor will be scheduled to run but will perform no work,
|
||||
as it will receive no FlowFile, and this leads to confusion as well. As a result, we can use the `@InputRequirement` annotation
|
||||
and provide it a value of `INPUT_REQUIRED`, `INPUT_ALLOWED`, or `INPUT_FORBIDDEN`. This provides information to the framework
|
||||
about when the Processor should be made invalid, or whether or not the user should even be able to draw a Connection to the
|
||||
Processor. For instance, if a Processor is annotated with `InputRequirement(Requirement.INPUT_FORBIDDEN)`, then the user will
|
||||
not even be able to create a Connection with that Processor as the destination.
|
||||
|
||||
|
||||
=== Data Buffering
|
||||
|
||||
|
|
|
@ -29,6 +29,8 @@ import org.apache.avro.file.DataFileStream;
|
|||
import org.apache.avro.generic.GenericData;
|
||||
import org.apache.avro.generic.GenericDatumReader;
|
||||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -46,6 +48,7 @@ import org.apache.nifi.processor.io.StreamCallback;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({ "json", "avro", "binary" })
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Converts a Binary Avro record into a JSON object. This processor provides a direct mapping of an Avro field to a JSON field, such "
|
||||
+ "that the resulting JSON will have the same hierarchical structure as the Avro document. Note that the Avro schema information will be lost, as this "
|
||||
+ "is not a translation from binary Avro to JSON formatted Avro. The output JSON is encoded the UTF-8 encoding. If an incoming FlowFile contains a stream of "
|
||||
|
|
|
@ -16,6 +16,19 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.avro;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.SchemaNormalization;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
|
@ -23,6 +36,8 @@ import org.apache.avro.generic.GenericDatumReader;
|
|||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -41,22 +56,10 @@ import org.apache.nifi.processor.exception.ProcessException;
|
|||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.NoSuchElementException;
|
||||
import java.util.Set;
|
||||
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({ "avro", "schema", "metadata" })
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Extracts metadata from the header of an Avro datafile.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "schema.type", description = "The type of the schema (i.e. record, enum, etc.)."),
|
||||
|
|
|
@ -16,6 +16,18 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.avro;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.avro.file.CodecFactory;
|
||||
import org.apache.avro.file.DataFileConstants;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
|
@ -26,6 +38,8 @@ import org.apache.avro.generic.GenericRecord;
|
|||
import org.apache.avro.io.DatumWriter;
|
||||
import org.apache.avro.io.Encoder;
|
||||
import org.apache.avro.io.EncoderFactory;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -45,21 +59,10 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({ "avro", "split" })
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Splits a binary encoded Avro datafile into smaller files based on the configured Output Size. The Output Strategy determines if " +
|
||||
"the smaller files will be Avro datafiles, or bare Avro records with metadata in the FlowFile attributes. The output will always be binary encoded.")
|
||||
public class SplitAvro extends AbstractProcessor {
|
||||
|
|
|
@ -24,6 +24,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -45,6 +47,7 @@ import com.amazonaws.services.s3.model.S3Object;
|
|||
|
||||
@SupportsBatching
|
||||
@SeeAlso({PutS3Object.class})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Amazon", "S3", "AWS", "Get", "Fetch"})
|
||||
@CapabilityDescription("Retrieves the contents of an S3 Object and writes it to the content of a FlowFile")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -28,6 +28,8 @@ import java.util.Map;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -54,6 +56,7 @@ import com.amazonaws.services.s3.model.StorageClass;
|
|||
|
||||
@SupportsBatching
|
||||
@SeeAlso({FetchS3Object.class})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Amazon", "S3", "AWS", "Archive", "Put"})
|
||||
@CapabilityDescription("Puts FlowFiles to an Amazon S3 Bucket")
|
||||
@DynamicProperty(name = "The name of a User-Defined Metadata field to add to the S3 Object",
|
||||
|
@ -101,6 +104,7 @@ public class PutS3Object extends AbstractS3Processor {
|
|||
.build();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void onTrigger(final ProcessContext context, final ProcessSession session) {
|
||||
FlowFile flowFile = session.get();
|
||||
if (flowFile == null) {
|
||||
|
|
|
@ -23,6 +23,8 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -41,6 +43,7 @@ import com.amazonaws.services.sns.model.PublishRequest;
|
|||
|
||||
@SupportsBatching
|
||||
@SeeAlso({GetSQS.class, PutSQS.class})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"amazon", "aws", "sns", "topic", "put", "publish", "pubsub"})
|
||||
@CapabilityDescription("Sends the content of a FlowFile as a notification to the Amazon Simple Notification Service")
|
||||
public class PutSNS extends AbstractSNSProcessor {
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.util.Arrays;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -37,6 +39,7 @@ import com.amazonaws.services.sqs.model.DeleteMessageBatchRequestEntry;
|
|||
|
||||
@SupportsBatching
|
||||
@SeeAlso({GetSQS.class, PutSQS.class})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Amazon", "AWS", "SQS", "Queue", "Delete"})
|
||||
@CapabilityDescription("Deletes a message from an Amazon Simple Queuing Service Queue")
|
||||
public class DeleteSQS extends AbstractSQSProcessor {
|
||||
|
|
|
@ -28,6 +28,8 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -51,8 +53,9 @@ import com.amazonaws.services.sqs.model.ReceiveMessageRequest;
|
|||
import com.amazonaws.services.sqs.model.ReceiveMessageResult;
|
||||
|
||||
@SupportsBatching
|
||||
@SeeAlso({ PutSQS.class, DeleteSQS.class })
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"Amazon", "AWS", "SQS", "Queue", "Get", "Fetch", "Poll"})
|
||||
@SeeAlso({PutSQS.class, DeleteSQS.class})
|
||||
@CapabilityDescription("Fetches messages from an Amazon Simple Queuing Service Queue")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "hash.value", description = "The MD5 sum of the message"),
|
||||
|
|
|
@ -28,6 +28,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -45,8 +47,9 @@ import com.amazonaws.services.sqs.model.SendMessageBatchRequest;
|
|||
import com.amazonaws.services.sqs.model.SendMessageBatchRequestEntry;
|
||||
|
||||
@SupportsBatching
|
||||
@SeeAlso({ GetSQS.class, DeleteSQS.class })
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Amazon", "AWS", "SQS", "Queue", "Put", "Publish"})
|
||||
@SeeAlso({GetSQS.class, DeleteSQS.class})
|
||||
@CapabilityDescription("Publishes a message to an Amazon Simple Queuing Service Queue")
|
||||
@DynamicProperty(name = "The name of a Message Attribute to add to the message", value = "The value of the Message Attribute",
|
||||
description = "Allows the user to add key/value pairs as Message Attributes by adding a property whose name will become the name of "
|
||||
|
|
|
@ -16,20 +16,19 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.flume;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.flume.EventDeliveryException;
|
||||
import org.apache.flume.Sink;
|
||||
import org.apache.flume.conf.Configurables;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
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.OnStopped;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
|
@ -40,12 +39,17 @@ import org.apache.nifi.processor.SchedulingContext;
|
|||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
/**
|
||||
* This processor runs a Flume sink
|
||||
*/
|
||||
@Tags({"flume", "hadoop", "put", "sink"})
|
||||
@CapabilityDescription("Execute a Flume sink. Each input FlowFile is converted into a Flume Event for processing by the sink.")
|
||||
@TriggerSerially
|
||||
@Tags({"flume", "hadoop", "put", "sink"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Execute a Flume sink. Each input FlowFile is converted into a Flume Event for processing by the sink.")
|
||||
public class ExecuteFlumeSink extends AbstractFlumeProcessor {
|
||||
|
||||
public static final PropertyDescriptor SINK_TYPE = new PropertyDescriptor.Builder()
|
||||
|
|
|
@ -16,12 +16,10 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.flume;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.flume.EventDeliveryException;
|
||||
import org.apache.flume.EventDrivenSource;
|
||||
import org.apache.flume.PollableSource;
|
||||
|
@ -29,12 +27,13 @@ import org.apache.flume.Source;
|
|||
import org.apache.flume.channel.ChannelProcessor;
|
||||
import org.apache.flume.conf.Configurables;
|
||||
import org.apache.flume.source.EventDrivenSourceRunner;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
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.OnStopped;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
|
@ -46,12 +45,17 @@ import org.apache.nifi.processor.SchedulingContext;
|
|||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
/**
|
||||
* This processor runs a Flume source
|
||||
*/
|
||||
@Tags({"flume", "hadoop", "get", "source"})
|
||||
@CapabilityDescription("Execute a Flume source. Each Flume Event is sent to the success relationship as a FlowFile")
|
||||
@TriggerSerially
|
||||
@Tags({"flume", "hadoop", "get", "source"})
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Execute a Flume source. Each Flume Event is sent to the success relationship as a FlowFile")
|
||||
public class ExecuteFlumeSource extends AbstractFlumeProcessor {
|
||||
|
||||
public static final PropertyDescriptor SOURCE_TYPE = new PropertyDescriptor.Builder()
|
||||
|
|
|
@ -20,6 +20,7 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.connectable.Connectable;
|
||||
import org.apache.nifi.controller.service.ControllerServiceNode;
|
||||
import org.apache.nifi.controller.service.ControllerServiceProvider;
|
||||
|
@ -48,6 +49,8 @@ public abstract class ProcessorNode extends AbstractConfiguredComponent implemen
|
|||
|
||||
public abstract boolean isHighThroughputSupported();
|
||||
|
||||
public abstract Requirement getInputRequirement();
|
||||
|
||||
@Override
|
||||
public abstract boolean isValid();
|
||||
|
||||
|
|
|
@ -16,14 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.controller;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
||||
import static java.util.Objects.requireNonNull;
|
||||
|
||||
import java.util.ArrayList;
|
||||
|
@ -43,6 +35,17 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||
|
||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.connectable.Connectable;
|
||||
|
@ -61,8 +64,6 @@ import org.apache.nifi.processor.Processor;
|
|||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.scheduling.SchedulingStrategy;
|
||||
import org.apache.nifi.util.FormatUtils;
|
||||
import org.apache.commons.lang3.builder.EqualsBuilder;
|
||||
import org.apache.commons.lang3.builder.HashCodeBuilder;
|
||||
import org.quartz.CronExpression;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
|
@ -108,6 +109,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
private final boolean triggerWhenAnyDestinationAvailable;
|
||||
private final boolean eventDrivenSupported;
|
||||
private final boolean batchSupported;
|
||||
private final Requirement inputRequirement;
|
||||
private final ValidationContextFactory validationContextFactory;
|
||||
private final ProcessScheduler processScheduler;
|
||||
private long runNanos = 0L;
|
||||
|
@ -156,6 +158,14 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
this.validationContextFactory = validationContextFactory;
|
||||
eventDrivenSupported = (procClass.isAnnotationPresent(EventDriven.class)
|
||||
|| procClass.isAnnotationPresent(org.apache.nifi.processor.annotation.EventDriven.class)) && !triggeredSerially && !triggerWhenEmpty;
|
||||
|
||||
final boolean inputRequirementPresent = procClass.isAnnotationPresent(InputRequirement.class);
|
||||
if (inputRequirementPresent) {
|
||||
inputRequirement = procClass.getAnnotation(InputRequirement.class).value();
|
||||
} else {
|
||||
inputRequirement = Requirement.INPUT_ALLOWED;
|
||||
}
|
||||
|
||||
schedulingStrategy = SchedulingStrategy.TIMER_DRIVEN;
|
||||
}
|
||||
|
||||
|
@ -977,6 +987,23 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
switch (getInputRequirement()) {
|
||||
case INPUT_ALLOWED:
|
||||
break;
|
||||
case INPUT_FORBIDDEN: {
|
||||
if (!getIncomingConnections().isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
break;
|
||||
}
|
||||
case INPUT_REQUIRED: {
|
||||
if (getIncomingConnections().isEmpty()) {
|
||||
return false;
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
return false;
|
||||
} finally {
|
||||
|
@ -1014,6 +1041,32 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
results.add(error);
|
||||
}
|
||||
}
|
||||
|
||||
switch (getInputRequirement()) {
|
||||
case INPUT_ALLOWED:
|
||||
break;
|
||||
case INPUT_FORBIDDEN: {
|
||||
final int incomingConnCount = getIncomingConnections().size();
|
||||
if (incomingConnCount != 0) {
|
||||
results.add(new ValidationResult.Builder()
|
||||
.explanation("Processor does not accept Incoming Connections but is currently configured with " + incomingConnCount + " Incoming Connections")
|
||||
.subject("Incoming Connections")
|
||||
.valid(false)
|
||||
.build());
|
||||
}
|
||||
break;
|
||||
}
|
||||
case INPUT_REQUIRED: {
|
||||
if (getIncomingConnections().isEmpty()) {
|
||||
results.add(new ValidationResult.Builder()
|
||||
.explanation("Processor required at least one Incoming Connection in order to perform its function but currently has no Incoming Connection")
|
||||
.subject("Incoming Connections")
|
||||
.valid(false)
|
||||
.build());
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
} catch (final Throwable t) {
|
||||
results.add(new ValidationResult.Builder().explanation("Failed to run validation due to " + t.toString()).valid(false).build());
|
||||
} finally {
|
||||
|
@ -1022,6 +1075,11 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
return results;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Requirement getInputRequirement() {
|
||||
return inputRequirement;
|
||||
}
|
||||
|
||||
/**
|
||||
* Establishes node equality (based on the processor's identifier)
|
||||
*
|
||||
|
@ -1112,7 +1170,7 @@ public class StandardProcessorNode extends ProcessorNode implements Connectable
|
|||
|
||||
@Override
|
||||
public Collection<ValidationResult> validate(final ValidationContext validationContext) {
|
||||
return processor.validate(validationContext);
|
||||
return getValidationErrors();
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -31,6 +31,8 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -59,6 +61,7 @@ import com.maxmind.geoip2.record.Subdivision;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"geo", "enrich", "ip", "maxmind"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Looks up geolocation information for an IP address and adds the geo information to FlowFile attributes. The "
|
||||
+ "geo data is provided as a MaxMind database. The attribute that contains the IP address to lookup is provided by the "
|
||||
+ "'IP Address Attribute' property. If the name of the attribute provided is 'X', then the the attributes added by enrichment "
|
||||
|
|
|
@ -23,7 +23,8 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.io.SequenceFile;
|
||||
import org.apache.hadoop.io.SequenceFile.CompressionType;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -57,6 +58,7 @@ import org.apache.nifi.processors.hadoop.util.SequenceFileWriter;
|
|||
*
|
||||
*/
|
||||
@SideEffectFree
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"hadoop", "sequence file", "create", "sequencefile"})
|
||||
@CapabilityDescription("Creates Hadoop Sequence Files from incoming flow files")
|
||||
@SeeAlso(PutHDFS.class)
|
||||
|
|
|
@ -29,6 +29,8 @@ import org.apache.hadoop.fs.FSDataInputStream;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.security.AccessControlException;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -44,6 +46,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"hadoop", "hdfs", "get", "ingest", "fetch", "source"})
|
||||
@CapabilityDescription("Retrieves a file from HDFS. The content of the incoming FlowFile is replaced by the content of the file in HDFS. "
|
||||
+ "The file in HDFS is left intact without any changes being made to it.")
|
||||
|
|
|
@ -41,6 +41,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.PathFilter;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.io.compress.CompressionCodecFactory;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -62,6 +64,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_ALLOWED)
|
||||
@Tags({"hadoop", "HDFS", "get", "fetch", "ingest", "source", "filesystem"})
|
||||
@CapabilityDescription("Fetch files from Hadoop Distributed File System (HDFS) into FlowFiles. This Processor will delete the file from HDFS after fetching it.")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -36,6 +36,8 @@ import org.apache.hadoop.fs.FileSystem;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsAction;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -66,6 +68,7 @@ import org.codehaus.jackson.map.ObjectMapper;
|
|||
|
||||
@TriggerSerially
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"hadoop", "HDFS", "get", "list", "ingest", "source", "filesystem"})
|
||||
@CapabilityDescription("Retrieves a listing of files from HDFS. For each file that is listed in HDFS, creates a FlowFile that represents "
|
||||
+ "the HDFS file so that it can be fetched in conjunction with ListHDFS. This Processor is designed to run on Primary Node only "
|
||||
|
|
|
@ -33,6 +33,8 @@ import org.apache.hadoop.fs.Path;
|
|||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.io.compress.CompressionCodec;
|
||||
import org.apache.hadoop.ipc.RemoteException;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -59,6 +61,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
/**
|
||||
* This processor copies FlowFiles to HDFS.
|
||||
*/
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"hadoop", "HDFS", "put", "copy", "filesystem"})
|
||||
@CapabilityDescription("Write FlowFile data to Hadoop Distributed File System (HDFS)")
|
||||
@WritesAttribute(attribute = "filename", description = "The name of the file written to HDFS comes from the value of this attribute.")
|
||||
|
|
|
@ -26,6 +26,8 @@ import java.util.List;
|
|||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -57,6 +59,7 @@ import ca.uhn.hl7v2.validation.impl.ValidationContextFactory;
|
|||
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"HL7", "health level 7", "healthcare", "extract", "attributes"})
|
||||
@CapabilityDescription("Extracts information from an HL7 (Health Level 7) formatted FlowFile and adds the information as FlowFile Attributes. "
|
||||
+ "The attributes are named as <Segment Name> <dot> <Field Index>. If the segment is repeating, the naming will be "
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.Set;
|
|||
import org.apache.nifi.annotation.behavior.DynamicProperties;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
@ -63,6 +65,7 @@ import ca.uhn.hl7v2.validation.impl.ValidationContextFactory;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"HL7", "healthcare", "route", "Health Level 7"})
|
||||
@DynamicProperties({
|
||||
@DynamicProperty(name = "Name of a Relationship", value = "An HL7 Query Language query",
|
||||
|
|
|
@ -16,11 +16,18 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.image;
|
||||
|
||||
import com.drew.imaging.ImageMetadataReader;
|
||||
import com.drew.imaging.ImageProcessingException;
|
||||
import com.drew.metadata.Directory;
|
||||
import com.drew.metadata.Metadata;
|
||||
import com.drew.metadata.Tag;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -30,25 +37,22 @@ import org.apache.nifi.components.PropertyDescriptor;
|
|||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
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.util.ObjectHolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.HashSet;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import com.drew.imaging.ImageMetadataReader;
|
||||
import com.drew.imaging.ImageProcessingException;
|
||||
import com.drew.metadata.Directory;
|
||||
import com.drew.metadata.Metadata;
|
||||
import com.drew.metadata.Tag;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Exif", "Exchangeable", "image", "file", "format", "JPG", "GIF", "PNG", "BMP", "metadata","IPTC", "XMP"})
|
||||
@CapabilityDescription("Extract the image metadata from flowfiles containing images. This processor relies on this "
|
||||
+ "metadata extractor library https://github.com/drewnoakes/metadata-extractor. It extracts a long list of "
|
||||
|
|
|
@ -17,7 +17,27 @@
|
|||
|
||||
package org.apache.nifi.processors.image;
|
||||
|
||||
import java.awt.Graphics2D;
|
||||
import java.awt.Image;
|
||||
import java.awt.image.BufferedImage;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import javax.imageio.ImageIO;
|
||||
import javax.imageio.ImageReader;
|
||||
import javax.imageio.stream.ImageInputStream;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -33,25 +53,9 @@ import org.apache.nifi.processor.io.StreamCallback;
|
|||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
import javax.imageio.ImageIO;
|
||||
import javax.imageio.ImageReader;
|
||||
import javax.imageio.stream.ImageInputStream;
|
||||
import java.awt.Image;
|
||||
import java.awt.Graphics2D;
|
||||
import java.awt.image.BufferedImage;
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Set;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Iterator;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@EventDriven
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({ "resize", "image", "jpg", "jpeg", "png", "bmp", "wbmp", "gif" })
|
||||
@CapabilityDescription("Resizes an image to user-specified dimensions. This Processor uses the image codecs registered with the "
|
||||
+ "environment that NiFi is running in. By default, this includes JPEG, PNG, BMP, WBMP, and GIF images.")
|
||||
|
|
|
@ -32,18 +32,13 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import kafka.consumer.Consumer;
|
||||
import kafka.consumer.ConsumerConfig;
|
||||
import kafka.consumer.ConsumerIterator;
|
||||
import kafka.consumer.KafkaStream;
|
||||
import kafka.javaapi.consumer.ConsumerConnector;
|
||||
import kafka.message.MessageAndMetadata;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
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.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
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.OnStopped;
|
||||
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
|
||||
|
@ -58,7 +53,15 @@ import org.apache.nifi.processor.exception.ProcessException;
|
|||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import kafka.consumer.Consumer;
|
||||
import kafka.consumer.ConsumerConfig;
|
||||
import kafka.consumer.ConsumerIterator;
|
||||
import kafka.consumer.KafkaStream;
|
||||
import kafka.javaapi.consumer.ConsumerConnector;
|
||||
import kafka.message.MessageAndMetadata;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Fetches messages from Apache Kafka")
|
||||
@Tags({"Kafka", "Apache", "Get", "Ingest", "Ingress", "Topic", "PubSub"})
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -30,10 +30,8 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import kafka.javaapi.producer.Producer;
|
||||
import kafka.producer.KeyedMessage;
|
||||
import kafka.producer.ProducerConfig;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -58,9 +56,13 @@ import org.apache.nifi.stream.io.StreamUtils;
|
|||
import org.apache.nifi.stream.io.util.NonThreadSafeCircularBuffer;
|
||||
import org.apache.nifi.util.LongHolder;
|
||||
|
||||
import kafka.javaapi.producer.Producer;
|
||||
import kafka.producer.KeyedMessage;
|
||||
import kafka.producer.ProducerConfig;
|
||||
import scala.actors.threadpool.Arrays;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({ "Apache", "Kafka", "Put", "Send", "Message", "PubSub" })
|
||||
@CapabilityDescription("Sends the contents of a FlowFile as a message to Apache Kafka")
|
||||
public class PutKafka extends AbstractProcessor {
|
||||
|
|
|
@ -18,18 +18,20 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kite;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import static org.apache.nifi.processor.util.StandardValidators.createLongValidator;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.CodecFactory;
|
||||
import org.apache.avro.file.DataFileWriter;
|
||||
import org.apache.avro.generic.GenericData.Record;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
|
@ -53,11 +55,13 @@ import org.kitesdk.data.spi.DefaultConfiguration;
|
|||
import org.kitesdk.data.spi.filesystem.CSVFileReader;
|
||||
import org.kitesdk.data.spi.filesystem.CSVProperties;
|
||||
|
||||
import static org.apache.nifi.processor.util.StandardValidators.createLongValidator;
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
@Tags({"kite", "csv", "avro"})
|
||||
@CapabilityDescription(
|
||||
"Converts CSV files to Avro according to an Avro Schema")
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Converts CSV files to Avro according to an Avro Schema")
|
||||
public class ConvertCSVToAvro extends AbstractKiteProcessor {
|
||||
|
||||
private static final CSVProperties DEFAULTS = new CSVProperties.Builder().build();
|
||||
|
|
|
@ -18,18 +18,18 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kite;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.CodecFactory;
|
||||
import org.apache.avro.file.DataFileWriter;
|
||||
import org.apache.avro.generic.GenericData.Record;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -47,9 +47,13 @@ import org.kitesdk.data.SchemaNotFoundException;
|
|||
import org.kitesdk.data.spi.DefaultConfiguration;
|
||||
import org.kitesdk.data.spi.filesystem.JSONFileReader;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
@Tags({"kite", "json", "avro"})
|
||||
@CapabilityDescription(
|
||||
"Converts JSON files to Avro according to an Avro Schema")
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Converts JSON files to Avro according to an Avro Schema")
|
||||
public class ConvertJSONToAvro extends AbstractKiteProcessor {
|
||||
|
||||
private static final Relationship SUCCESS = new Relationship.Builder()
|
||||
|
|
|
@ -18,16 +18,17 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.kite;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.avro.Schema;
|
||||
import org.apache.avro.file.DataFileStream;
|
||||
import org.apache.avro.generic.GenericData.Record;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -46,6 +47,10 @@ import org.kitesdk.data.ValidationException;
|
|||
import org.kitesdk.data.View;
|
||||
import org.kitesdk.data.spi.SchemaValidationUtil;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"kite", "avro", "parquet", "hadoop", "hive", "hdfs", "hbase"})
|
||||
@CapabilityDescription("Stores Avro records in a Kite dataset")
|
||||
public class StoreInKiteDataset extends AbstractKiteProcessor {
|
||||
|
|
|
@ -33,6 +33,8 @@ import javax.ws.rs.core.MediaType;
|
|||
import javax.ws.rs.core.MultivaluedMap;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -70,6 +72,7 @@ import com.sun.jersey.api.json.JSONConfiguration;
|
|||
import com.sun.jersey.core.util.MultivaluedMapImpl;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"yandex", "translate", "translation", "language"})
|
||||
@CapabilityDescription("Translates content and attributes from one language to another")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -0,0 +1 @@
|
|||
/bin/
|
|
@ -32,6 +32,8 @@ import java.util.concurrent.BlockingQueue;
|
|||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -57,8 +59,8 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import com.twitter.hbc.ClientBuilder;
|
||||
import com.twitter.hbc.core.Client;
|
||||
import com.twitter.hbc.core.Constants;
|
||||
import com.twitter.hbc.core.endpoint.Location.Coordinate ;
|
||||
import com.twitter.hbc.core.endpoint.Location ;
|
||||
import com.twitter.hbc.core.endpoint.Location.Coordinate ;
|
||||
import com.twitter.hbc.core.endpoint.StatusesFilterEndpoint;
|
||||
import com.twitter.hbc.core.endpoint.StatusesFirehoseEndpoint;
|
||||
import com.twitter.hbc.core.endpoint.StatusesSampleEndpoint;
|
||||
|
@ -69,6 +71,7 @@ import com.twitter.hbc.httpclient.auth.Authentication;
|
|||
import com.twitter.hbc.httpclient.auth.OAuth1;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"twitter", "tweets", "social media", "status", "json"})
|
||||
@CapabilityDescription("Pulls status changes from Twitter's streaming API")
|
||||
@WritesAttribute(attribute = "mime.type", description = "Sets mime type to application/json")
|
||||
|
|
|
@ -18,7 +18,29 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.solr;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnRemoved;
|
||||
|
@ -41,27 +63,8 @@ import org.apache.solr.client.solrj.util.ClientUtils;
|
|||
import org.apache.solr.common.SolrDocument;
|
||||
import org.apache.solr.common.SolrDocumentList;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.text.SimpleDateFormat;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Date;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Locale;
|
||||
import java.util.Properties;
|
||||
import java.util.Set;
|
||||
import java.util.TimeZone;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
@Tags({"Apache", "Solr", "Get", "Pull"})
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Queries Solr and outputs the results as a FlowFile")
|
||||
public class GetSolr extends SolrProcessor {
|
||||
|
||||
|
|
|
@ -18,7 +18,24 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.solr;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -40,22 +57,8 @@ import org.apache.solr.common.params.ModifiableSolrParams;
|
|||
import org.apache.solr.common.params.MultiMapSolrParams;
|
||||
import org.apache.solr.common.util.ContentStreamBase;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.SortedMap;
|
||||
import java.util.TreeMap;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
@Tags({"Apache", "Solr", "Put", "Send"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Sends the contents of a FlowFile as a ContentStream to Solr")
|
||||
@DynamicProperty(name="A Solr request parameter name", value="A Solr request parameter value",
|
||||
description="These parameters will be passed to Solr on the request")
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.concurrent.TimeUnit;
|
|||
import org.apache.commons.codec.binary.Base64InputStream;
|
||||
import org.apache.commons.codec.binary.Base64OutputStream;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -51,6 +53,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
@SupportsBatching
|
||||
@Tags({"encode", "base64"})
|
||||
@CapabilityDescription("Encodes or decodes content to and from base64")
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
public class Base64EncodeContent extends AbstractProcessor {
|
||||
|
||||
public static final String ENCODE_MODE = "Encode";
|
||||
|
|
|
@ -29,20 +29,18 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import lzma.sdk.lzma.Decoder;
|
||||
import lzma.streams.LzmaInputStream;
|
||||
import lzma.streams.LzmaOutputStream;
|
||||
|
||||
import org.apache.commons.compress.compressors.CompressorStreamFactory;
|
||||
import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream;
|
||||
import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
|
@ -63,9 +61,14 @@ import org.tukaani.xz.LZMA2Options;
|
|||
import org.tukaani.xz.XZInputStream;
|
||||
import org.tukaani.xz.XZOutputStream;
|
||||
|
||||
import lzma.sdk.lzma.Decoder;
|
||||
import lzma.streams.LzmaInputStream;
|
||||
import lzma.streams.LzmaOutputStream;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"content", "compress", "decompress", "gzip", "bzip2", "lzma", "xz-lzma2"})
|
||||
@CapabilityDescription("Compresses or decompresses the contents of FlowFiles using a user-specified compression algorithm and updates the mime.type "
|
||||
+ "attribute as appropriate")
|
||||
|
|
|
@ -31,6 +31,12 @@ import java.util.concurrent.atomic.AtomicLong;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -43,10 +49,6 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.timebuffer.EntityAccess;
|
||||
|
@ -54,6 +56,7 @@ import org.apache.nifi.util.timebuffer.TimedBuffer;
|
|||
|
||||
@SideEffectFree
|
||||
@TriggerSerially
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"rate control", "throttle", "rate", "throughput"})
|
||||
@CapabilityDescription("Controls the rate at which data is transferred to follow-on processors.")
|
||||
public class ControlRate extends AbstractProcessor {
|
||||
|
|
|
@ -33,8 +33,10 @@ import org.apache.nifi.flowfile.FlowFile;
|
|||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
|
@ -76,6 +78,7 @@ import java.util.concurrent.TimeUnit;
|
|||
*/
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@SupportsBatching
|
||||
@Tags({"text", "convert", "characterset", "character set"})
|
||||
@CapabilityDescription("Converts a FlowFile's content from one character set to another")
|
||||
|
|
|
@ -34,10 +34,12 @@ import java.util.Map;
|
|||
import java.util.Set;
|
||||
import java.util.UUID;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -64,6 +66,7 @@ import org.codehaus.jackson.node.JsonNodeFactory;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@SeeAlso(PutSQL.class)
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"json", "sql", "database", "rdbms", "insert", "update", "relational", "flat"})
|
||||
@CapabilityDescription("Converts a JSON-formatted FlowFile into an UPDATE or INSERT SQL statement. The incoming FlowFile is expected to be "
|
||||
+ "\"flat\" JSON message, meaning that it consists of a single JSON element and each field maps to a simple type. If a field maps to "
|
||||
|
|
|
@ -28,11 +28,13 @@ import java.util.concurrent.TimeUnit;
|
|||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.distributed.cache.client.Deserializer;
|
||||
import org.apache.nifi.distributed.cache.client.DistributedMapCacheClient;
|
||||
|
@ -52,6 +54,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@EventDriven
|
||||
@SupportsBatching
|
||||
@Tags({"hash", "dupe", "duplicate", "dedupe"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Caches a value, computed from FlowFile attributes, for each incoming FlowFile and determines if the cached value has already been seen. "
|
||||
+ "If so, routes the FlowFile to 'duplicate' with an attribute named 'original.identifier' that specifies the original FlowFile's"
|
||||
+ "\"description\", which is specified in the <FlowFile Description> property. If the FlowFile is not determined to be a duplicate, the Processor "
|
||||
|
|
|
@ -32,9 +32,11 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenAnyDestinationAvailable;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.DynamicRelationship;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -57,6 +59,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@TriggerWhenAnyDestinationAvailable
|
||||
@Tags({"distribute", "load balance", "route", "round robin", "weighted"})
|
||||
@CapabilityDescription("Distributes FlowFiles to downstream processors based on a Distribution Strategy. If using the Round Robin "
|
||||
|
|
|
@ -21,7 +21,9 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -36,6 +38,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@EventDriven
|
||||
@SupportsBatching
|
||||
@Tags({"test", "load", "duplicate"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Intended for load testing, this processor will create the configured number of copies of each incoming FlowFile")
|
||||
public class DuplicateFlowFile extends AbstractProcessor {
|
||||
|
||||
|
|
|
@ -26,13 +26,20 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.commons.codec.DecoderException;
|
||||
import org.apache.commons.codec.binary.Base32InputStream;
|
||||
import org.apache.commons.codec.binary.Base32OutputStream;
|
||||
|
||||
import org.apache.commons.codec.binary.Base64InputStream;
|
||||
import org.apache.commons.codec.binary.Base64OutputStream;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -41,11 +48,6 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.processors.standard.util.ValidatingBase32InputStream;
|
||||
import org.apache.nifi.processors.standard.util.ValidatingBase64InputStream;
|
||||
|
@ -55,6 +57,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"encode", "decode", "base64", "hex"})
|
||||
@CapabilityDescription("Encodes the FlowFile content in base64")
|
||||
public class EncodeContent extends AbstractProcessor {
|
||||
|
|
|
@ -27,6 +27,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -55,6 +57,7 @@ import org.bouncycastle.jce.provider.BouncyCastleProvider;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"encryption", "decryption", "password", "JCE", "OpenPGP", "PGP", "GPG"})
|
||||
@CapabilityDescription("Encrypts or Decrypts a FlowFile using either symmetric encryption with a password and randomly generated salt, or asymmetric encryption using a public and secret key.")
|
||||
public class EncryptContent extends AbstractProcessor {
|
||||
|
|
|
@ -16,13 +16,25 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.InvalidJsonException;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.PathNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -42,24 +54,16 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
|
|||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.ConcurrentMap;
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.InvalidJsonException;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.PathNotFoundException;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"JSON", "evaluate", "JsonPath"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Evaluates one or more JsonPath expressions against the content of a FlowFile. "
|
||||
+ "The results of those expressions are assigned to FlowFile Attributes or are written to the content of the FlowFile itself, "
|
||||
+ "depending on configuration of the Processor. "
|
||||
|
|
|
@ -49,40 +49,43 @@ import javax.xml.xpath.XPathExpressionException;
|
|||
import javax.xml.xpath.XPathFactory;
|
||||
import javax.xml.xpath.XPathFactoryConfigurationException;
|
||||
|
||||
import net.sf.saxon.lib.NamespaceConstant;
|
||||
import net.sf.saxon.xpath.XPathEvaluator;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import net.sf.saxon.lib.NamespaceConstant;
|
||||
import net.sf.saxon.xpath.XPathEvaluator;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"XML", "evaluate", "XPath"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Evaluates one or more XPaths against the content of a FlowFile. The results of those XPaths are assigned to "
|
||||
+ "FlowFile Attributes or are written to the content of the FlowFile itself, depending on configuration of the "
|
||||
+ "Processor. XPaths are entered by adding user-defined properties; the name of the property maps to the Attribute "
|
||||
|
|
|
@ -40,23 +40,15 @@ import javax.xml.transform.TransformerFactoryConfigurationError;
|
|||
import javax.xml.transform.sax.SAXSource;
|
||||
import javax.xml.transform.stream.StreamResult;
|
||||
|
||||
import net.sf.saxon.s9api.DOMDestination;
|
||||
import net.sf.saxon.s9api.Processor;
|
||||
import net.sf.saxon.s9api.SaxonApiException;
|
||||
import net.sf.saxon.s9api.XQueryCompiler;
|
||||
import net.sf.saxon.s9api.XQueryEvaluator;
|
||||
import net.sf.saxon.s9api.XQueryExecutable;
|
||||
import net.sf.saxon.s9api.XdmItem;
|
||||
import net.sf.saxon.s9api.XdmNode;
|
||||
import net.sf.saxon.s9api.XdmValue;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -78,10 +70,21 @@ import org.apache.nifi.util.ObjectHolder;
|
|||
import org.w3c.dom.Document;
|
||||
import org.xml.sax.InputSource;
|
||||
|
||||
import net.sf.saxon.s9api.DOMDestination;
|
||||
import net.sf.saxon.s9api.Processor;
|
||||
import net.sf.saxon.s9api.SaxonApiException;
|
||||
import net.sf.saxon.s9api.XQueryCompiler;
|
||||
import net.sf.saxon.s9api.XQueryEvaluator;
|
||||
import net.sf.saxon.s9api.XQueryExecutable;
|
||||
import net.sf.saxon.s9api.XdmItem;
|
||||
import net.sf.saxon.s9api.XdmNode;
|
||||
import net.sf.saxon.s9api.XdmValue;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"XML", "evaluate", "XPath", "XQuery"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription(
|
||||
"Evaluates one or more XQueries against the content of a FlowFile. The results of those XQueries are assigned "
|
||||
+ "to FlowFile Attributes or are written to the content of the FlowFile itself, depending on configuration of "
|
||||
|
|
|
@ -43,6 +43,8 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
|
@ -60,6 +62,7 @@ import org.apache.nifi.processor.io.OutputStreamCallback;
|
|||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.ArgumentUtils;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"command", "process", "source", "external", "invoke", "script"})
|
||||
@CapabilityDescription("Runs an operating system command specified by the user and writes the output of that command to a FlowFile. If the command is expected "
|
||||
+ "to be long-running, the Processor can output the partial data on a specified interval. When this option is used, the output is expected to be in textual "
|
||||
|
|
|
@ -30,6 +30,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -48,6 +50,7 @@ import org.apache.nifi.util.LongHolder;
|
|||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@EventDriven
|
||||
@InputRequirement(Requirement.INPUT_ALLOWED)
|
||||
@Tags({ "sql", "select", "jdbc", "query", "database" })
|
||||
@CapabilityDescription("Execute provided SQL select query. Query result will be converted to Avro format."
|
||||
+ " Streaming is used so arbitrarily large result sets are supported. This processor can be scheduled to run on " +
|
||||
|
|
|
@ -35,11 +35,13 @@ import org.apache.commons.io.IOUtils;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -119,6 +121,7 @@ import org.apache.nifi.stream.io.StreamUtils;
|
|||
*/
|
||||
@EventDriven
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"command execution", "command", "stream", "execute"})
|
||||
@CapabilityDescription("Executes an external command on the contents of a flow file, and creates a new flow file with the results of the command.")
|
||||
@DynamicProperty(name = "An environment variable name", value = "An environment variable value", description = "These environment variables are passed to the process spawned by this Processor")
|
||||
|
|
|
@ -34,6 +34,8 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -56,6 +58,7 @@ import org.apache.nifi.stream.io.StreamUtils;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"evaluate", "extract", "Text", "Regular Expression", "regex"})
|
||||
@CapabilityDescription(
|
||||
"Evaluates one or more Regular Expressions against the content of a FlowFile. "
|
||||
|
|
|
@ -26,6 +26,12 @@ import java.util.Random;
|
|||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
|
@ -34,15 +40,12 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
@SupportsBatching
|
||||
@Tags({"test", "random", "generate"})
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("This processor creates FlowFiles of random data and is used for load testing")
|
||||
public class GenerateFlowFile extends AbstractProcessor {
|
||||
|
||||
|
|
|
@ -20,17 +20,20 @@ import java.util.ArrayList;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processors.standard.util.FTPTransfer;
|
||||
import org.apache.nifi.processors.standard.util.FileTransfer;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"FTP", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"})
|
||||
@CapabilityDescription("Fetches files from an FTP Server and creates FlowFiles from them")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -49,12 +49,14 @@ import java.util.concurrent.locks.Lock;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
|
@ -70,6 +72,7 @@ import org.apache.nifi.processor.exception.ProcessException;
|
|||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"local", "files", "filesystem", "ingest", "ingress", "get", "source", "input"})
|
||||
@CapabilityDescription("Creates FlowFiles from files in a directory. NiFi will ignore files it doesn't have at least read permissions for.")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -69,6 +69,8 @@ import org.apache.http.conn.ssl.TrustSelfSignedStrategy;
|
|||
import org.apache.http.impl.client.BasicCredentialsProvider;
|
||||
import org.apache.http.impl.client.HttpClientBuilder;
|
||||
import org.apache.http.impl.conn.BasicHttpClientConnectionManager;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -94,6 +96,7 @@ import org.apache.nifi.ssl.SSLContextService.ClientAuth;
|
|||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@Tags({"get", "fetch", "poll", "http", "https", "ingest", "source", "input"})
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Fetches a file via HTTP")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "filename", description = "The filename is set to the name of the file on the remote server"),
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.util.concurrent.LinkedBlockingQueue;
|
|||
|
||||
import javax.jms.JMSException;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -34,6 +36,7 @@ import org.apache.nifi.processors.standard.util.JmsFactory;
|
|||
import org.apache.nifi.processors.standard.util.WrappedMessageConsumer;
|
||||
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"jms", "queue", "listen", "get", "pull", "source", "consume", "consumer"})
|
||||
@CapabilityDescription("Pulls messages from a JMS Queue, creating a FlowFile for each JMS Message or bundle of messages, as configured")
|
||||
@SeeAlso(PutJMS.class)
|
||||
|
|
|
@ -41,6 +41,8 @@ import javax.jms.InvalidDestinationException;
|
|||
import javax.jms.JMSException;
|
||||
import javax.jms.Session;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -60,6 +62,7 @@ import org.apache.nifi.processors.standard.util.WrappedMessageConsumer;
|
|||
|
||||
@TriggerSerially
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"jms", "topic", "subscription", "durable", "non-durable", "listen", "get", "pull", "source", "consume", "consumer"})
|
||||
@CapabilityDescription("Pulls messages from a JMS Topic, creating a FlowFile for each JMS Message or bundle of messages, as configured")
|
||||
@SeeAlso(PutJMS.class)
|
||||
|
|
|
@ -21,11 +21,13 @@ import java.util.Collection;
|
|||
import java.util.Collections;
|
||||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -34,6 +36,7 @@ import org.apache.nifi.processor.ProcessorInitializationContext;
|
|||
import org.apache.nifi.processors.standard.util.FileTransfer;
|
||||
import org.apache.nifi.processors.standard.util.SFTPTransfer;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"sftp", "get", "retrieve", "files", "fetch", "remote", "ingest", "source", "input"})
|
||||
@CapabilityDescription("Fetches files from an SFTP Server and creates FlowFiles from them")
|
||||
@WritesAttributes({
|
||||
|
|
|
@ -44,11 +44,13 @@ import javax.servlet.http.HttpServletRequest;
|
|||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
|
@ -75,6 +77,7 @@ import org.eclipse.jetty.util.ssl.SslContextFactory;
|
|||
|
||||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"http", "https", "request", "listen", "ingress", "web service"})
|
||||
@CapabilityDescription("Starts an HTTP Server and listens for HTTP Requests. For each request, creates a FlowFile and transfers to 'success'. "
|
||||
+ "This Processor is designed to be used in conjunction with the HandleHttpResponse Processor in order to create a Web Service")
|
||||
|
|
|
@ -27,8 +27,10 @@ import java.util.regex.Pattern;
|
|||
import javax.servlet.http.HttpServletResponse;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
|
@ -41,6 +43,7 @@ import org.apache.nifi.processor.Relationship;
|
|||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"http", "https", "response", "egress", "web service"})
|
||||
@CapabilityDescription("Sends an HTTP Response to the Requestor that generated a FlowFile. This Processor is designed to be used in conjunction with "
|
||||
+ "the HandleHttpRequest in order to create a web service.")
|
||||
|
|
|
@ -33,11 +33,13 @@ import org.apache.commons.codec.digest.DigestUtils;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -96,6 +98,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"attributes", "hash"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Hashes together the key/value pairs of several FlowFile Attributes and adds the hash as a new attribute. "
|
||||
+ "Optional properties are to be added such that the name of the property is the name of a FlowFile Attribute to consider "
|
||||
+ "and the value of the property is a regular expression that, if matched by the attribute value, will cause that attribute "
|
||||
|
|
|
@ -29,10 +29,12 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -50,6 +52,7 @@ import org.apache.nifi.util.ObjectHolder;
|
|||
|
||||
@EventDriven
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"hash", "content", "MD5", "SHA-1", "SHA-256"})
|
||||
@CapabilityDescription("Calculates a hash value for the Content of a FlowFile and puts that hash value on the FlowFile as an attribute whose name "
|
||||
+ "is determined by the <Hash Attribute Name> property")
|
||||
|
|
|
@ -24,11 +24,13 @@ import java.util.HashSet;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -65,6 +67,7 @@ import org.apache.tika.mime.MimeTypeException;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"compression", "gzip", "bzip2", "zip", "MIME", "mime.type", "file", "identify"})
|
||||
@CapabilityDescription("Attempts to identify the MIME Type used for a FlowFile. If the MIME Type can be identified, "
|
||||
+ "an attribute with the name 'mime.type' is added with the value being the MIME Type. If the MIME Type cannot be determined, "
|
||||
|
|
|
@ -54,6 +54,8 @@ import javax.net.ssl.SSLSession;
|
|||
import org.apache.commons.codec.binary.Base64;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
|
@ -77,6 +79,7 @@ import org.joda.time.format.DateTimeFormatter;
|
|||
|
||||
@SupportsBatching
|
||||
@Tags({"http", "https", "rest", "client"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("An HTTP client processor which converts FlowFile attributes to HTTP headers, with configurable HTTP method, url, etc.")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "invokehttp.status.code", description = "The status code that is returned"),
|
||||
|
|
|
@ -31,10 +31,14 @@ import java.util.regex.Pattern;
|
|||
import javax.servlet.Servlet;
|
||||
import javax.ws.rs.Path;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.LeakyBucketStreamThrottler;
|
||||
import org.apache.nifi.stream.io.StreamThrottler;
|
||||
import org.apache.nifi.processor.AbstractSessionFactoryProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
|
@ -42,15 +46,12 @@ import org.apache.nifi.processor.ProcessSession;
|
|||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.servlets.ContentAcknowledgmentServlet;
|
||||
import org.apache.nifi.processors.standard.servlets.ListenHTTPServlet;
|
||||
import org.apache.nifi.ssl.SSLContextService;
|
||||
|
||||
import org.apache.nifi.stream.io.LeakyBucketStreamThrottler;
|
||||
import org.apache.nifi.stream.io.StreamThrottler;
|
||||
import org.eclipse.jetty.server.Connector;
|
||||
import org.eclipse.jetty.server.HttpConfiguration;
|
||||
import org.eclipse.jetty.server.HttpConnectionFactory;
|
||||
|
@ -62,6 +63,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler;
|
|||
import org.eclipse.jetty.util.ssl.SslContextFactory;
|
||||
import org.eclipse.jetty.util.thread.QueuedThreadPool;
|
||||
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@Tags({"ingest", "http", "https", "rest", "listen"})
|
||||
@CapabilityDescription("Starts an HTTP Server that is used to receive FlowFiles from remote sources. The URL of the Service will be http://{hostname}:{port}/contentListener")
|
||||
public class ListenHTTP extends AbstractSessionFactoryProcessor {
|
||||
|
|
|
@ -41,6 +41,15 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import java.util.concurrent.locks.Lock;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
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.OnStopped;
|
||||
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
|
@ -58,19 +67,11 @@ import org.apache.nifi.processor.ProcessContext;
|
|||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.ProcessSessionFactory;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.lifecycle.OnStopped;
|
||||
import org.apache.nifi.annotation.lifecycle.OnUnscheduled;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.UDPStreamConsumer;
|
||||
import org.apache.nifi.util.Tuple;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* This processor listens for Datagram Packets on a given port and concatenates the contents of those packets together generating flow files roughly as often as the internal buffer fills up or until
|
||||
|
@ -113,6 +114,7 @@ import org.apache.commons.lang3.StringUtils;
|
|||
*/
|
||||
@TriggerWhenEmpty
|
||||
@Tags({"ingest", "udp", "listen", "source"})
|
||||
@InputRequirement(Requirement.INPUT_FORBIDDEN)
|
||||
@CapabilityDescription("Listens for Datagram Packets on a given port and concatenates the contents of those packets "
|
||||
+ "together generating flow files")
|
||||
public class ListenUDP extends AbstractSessionFactoryProcessor {
|
||||
|
|
|
@ -27,6 +27,14 @@ import java.util.List;
|
|||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -35,22 +43,16 @@ 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.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.eclipse.jetty.util.StringUtil;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"attributes", "logging"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
public class LogAttribute extends AbstractProcessor {
|
||||
|
||||
public static final PropertyDescriptor LOG_LEVEL = new PropertyDescriptor.Builder()
|
||||
|
|
|
@ -48,15 +48,17 @@ import org.apache.avro.generic.GenericDatumWriter;
|
|||
import org.apache.avro.generic.GenericRecord;
|
||||
import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
|
||||
import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttributes;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.PropertyValue;
|
||||
|
@ -86,6 +88,7 @@ import org.apache.nifi.util.ObjectHolder;
|
|||
|
||||
@SideEffectFree
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"merge", "content", "correlation", "tar", "zip", "stream", "concatenation", "archive", "flowfile-stream", "flowfile-stream-v3"})
|
||||
@CapabilityDescription("Merges a Group of FlowFiles together based on a user-defined strategy and packages them into a single FlowFile. "
|
||||
+ "It is recommended that the Processor be configured with only a single incoming connection, as Group of FlowFiles will not be "
|
||||
|
|
|
@ -25,28 +25,32 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@Tags({"binary", "discard", "keep"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Keep or discard bytes range from a binary file.")
|
||||
public class ModifyBytes extends AbstractProcessor {
|
||||
|
||||
|
|
|
@ -16,6 +16,22 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
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.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.TriggerSerially;
|
||||
import org.apache.nifi.annotation.behavior.TriggerWhenEmpty;
|
||||
|
@ -36,23 +52,10 @@ import org.apache.nifi.processor.Relationship;
|
|||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.Charset;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
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.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
@SideEffectFree
|
||||
@TriggerSerially
|
||||
@TriggerWhenEmpty
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"monitor", "flow", "active", "inactive", "activity", "detection"})
|
||||
@CapabilityDescription("Monitors the flow for activity and sends out an indicator when the flow has not had any data for "
|
||||
+ "some specified amount of time and again when the flow's activity is restored")
|
||||
|
|
|
@ -82,6 +82,8 @@ import org.apache.http.impl.conn.PoolingHttpClientConnectionManager;
|
|||
import org.apache.http.protocol.HttpContext;
|
||||
import org.apache.http.protocol.HttpCoreContext;
|
||||
import org.apache.http.util.EntityUtils;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -121,6 +123,7 @@ import org.apache.nifi.util.StopWatch;
|
|||
import com.sun.jersey.api.client.ClientResponse.Status;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"http", "https", "remote", "copy", "archive"})
|
||||
@CapabilityDescription("Performs an HTTP Post with the content of the FlowFile")
|
||||
@ReadsAttribute(attribute = "mime.type", description = "If not sending data as a FlowFile, the mime.type attribute will be used to set the HTTP Header for Content-Type")
|
||||
|
|
|
@ -47,6 +47,8 @@ import javax.mail.internet.PreencodedMimeBodyPart;
|
|||
import javax.mail.util.ByteArrayDataSource;
|
||||
|
||||
import org.apache.commons.codec.binary.Base64;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -67,6 +69,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
|
||||
@SupportsBatching
|
||||
@Tags({"email", "put", "notify", "smtp"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Sends an e-mail to configured recipients for each incoming FlowFile")
|
||||
public class PutEmail extends AbstractProcessor {
|
||||
|
||||
|
|
|
@ -28,6 +28,8 @@ import java.util.regex.Pattern;
|
|||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperties;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -41,6 +43,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.processors.standard.util.FTPTransfer;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"remote", "copy", "egress", "put", "ftp", "archive", "files"})
|
||||
@CapabilityDescription("Sends FlowFiles to an FTP Server")
|
||||
@SeeAlso(GetFTP.class)
|
||||
|
|
|
@ -34,6 +34,8 @@ import java.util.Set;
|
|||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -52,6 +54,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"put", "local", "copy", "archive", "files", "filesystem"})
|
||||
@CapabilityDescription("Writes the contents of a FlowFile to the local file system")
|
||||
@SeeAlso(GetFile.class)
|
||||
|
|
|
@ -39,9 +39,9 @@ import static org.apache.nifi.processors.standard.util.JmsProperties.MESSAGE_TTL
|
|||
import static org.apache.nifi.processors.standard.util.JmsProperties.MESSAGE_TYPE;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_BYTE;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_EMPTY;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_MAP;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_STREAM;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_TEXT;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.MSG_TYPE_MAP;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.PASSWORD;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.REPLY_TO_QUEUE;
|
||||
import static org.apache.nifi.processors.standard.util.JmsProperties.TIMEOUT;
|
||||
|
@ -70,6 +70,8 @@ import javax.jms.MessageProducer;
|
|||
import javax.jms.Session;
|
||||
import javax.jms.StreamMessage;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
|
@ -89,6 +91,7 @@ import org.apache.nifi.processors.standard.util.WrappedMessageProducer;
|
|||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
|
||||
@Tags({"jms", "send", "put"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Creates a JMS Message from the contents of a FlowFile and sends the message to a JMS Server")
|
||||
@SeeAlso({GetJMSQueue.class, GetJMSTopic.class})
|
||||
public class PutJMS extends AbstractProcessor {
|
||||
|
|
|
@ -21,6 +21,8 @@ import java.util.Collections;
|
|||
import java.util.List;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
|
@ -31,6 +33,7 @@ import org.apache.nifi.processor.ProcessorInitializationContext;
|
|||
import org.apache.nifi.processors.standard.util.SFTPTransfer;
|
||||
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"remote", "copy", "egress", "put", "sftp", "archive", "files"})
|
||||
@CapabilityDescription("Sends FlowFiles to an SFTP Server")
|
||||
@SeeAlso(GetSFTP.class)
|
||||
|
|
|
@ -45,6 +45,8 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttributes;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
|
@ -68,6 +70,7 @@ import org.apache.nifi.stream.io.StreamUtils;
|
|||
|
||||
@SupportsBatching
|
||||
@SeeAlso(ConvertJSONToSQL.class)
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"sql", "put", "rdbms", "database", "update", "insert", "relational"})
|
||||
@CapabilityDescription("Executes a SQL UPDATE or INSERT command. The content of an incoming FlowFile is expected to be the SQL command "
|
||||
+ "to execute. The SQL command may use the ? to escape parameters. In this case, the parameters to use must exist as FlowFile attributes "
|
||||
|
|
|
@ -16,33 +16,7 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.processor.ProcessContext;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.ProcessorInitializationContext;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
import org.apache.nifi.processor.ProcessSession;
|
||||
import org.apache.nifi.processor.Relationship;
|
||||
import java.io.BufferedWriter;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.processor.util.FlowFileFilters;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.NLKBufferedReader;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.InputStreamReader;
|
||||
|
@ -58,9 +32,37 @@ import java.util.concurrent.TimeUnit;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.Validator;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
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.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.processor.util.FlowFileFilters;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.NLKBufferedReader;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex"})
|
||||
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of "
|
||||
+ "the content that matches the Regular Expression with some alternate value.")
|
||||
|
|
|
@ -39,12 +39,19 @@ import java.util.concurrent.locks.ReentrantLock;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
import org.apache.nifi.components.ValidationResult;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
|
@ -52,21 +59,16 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.io.StreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.util.StopWatch;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Text", "Regular Expression", "Update", "Change", "Replace", "Modify", "Regex", "Mapping"})
|
||||
@CapabilityDescription("Updates the content of a FlowFile by evaluating a Regular Expression against it and replacing the section of the content that "
|
||||
+ "matches the Regular Expression with some alternate value provided in a mapping file.")
|
||||
|
|
|
@ -29,6 +29,8 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.DynamicRelationship;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -59,6 +61,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"attributes", "routing", "Attribute Expression Language", "regexp", "regex", "Regular Expression", "Expression Language"})
|
||||
@CapabilityDescription("Routes FlowFiles based on their Attributes using the Attribute Expression Language")
|
||||
@DynamicProperty(name = "Relationship Name", value = "Attribute Expression Language", supportsExpressionLanguage = true, description = "Routes FlowFiles whose "
|
||||
|
|
|
@ -29,10 +29,18 @@ import java.util.Set;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.DynamicRelationship;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.expression.AttributeValueDecorator;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.StreamUtils;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
import org.apache.nifi.processor.DataUnit;
|
||||
|
@ -40,20 +48,15 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.DynamicRelationship;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
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.IntegerHolder;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"route", "content", "regex", "regular expression", "regexp"})
|
||||
@CapabilityDescription("Applies Regular Expressions to the content of a FlowFile and routes a copy of the FlowFile to each "
|
||||
+ "destination whose Regular Expression matches. Regular Expressions are added as User-Defined Properties where the name "
|
||||
|
|
|
@ -32,28 +32,31 @@ import java.util.Set;
|
|||
import java.util.regex.Matcher;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.util.file.monitor.LastModifiedMonitor;
|
||||
import org.apache.nifi.util.file.monitor.SynchronousFileWatcher;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.exception.ProcessException;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.file.monitor.LastModifiedMonitor;
|
||||
import org.apache.nifi.util.file.monitor.SynchronousFileWatcher;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"scan", "attributes", "search", "lookup"})
|
||||
@CapabilityDescription("Scans the specified attributes of FlowFiles, checking to see if any of their values are "
|
||||
+ "present within the specified dictionary of terms")
|
||||
|
|
|
@ -35,11 +35,13 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -63,6 +65,7 @@ import org.apache.nifi.util.search.ahocorasick.SearchState;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"aho-corasick", "scan", "content", "byte sequence", "search", "find", "dictionary"})
|
||||
@CapabilityDescription("Scans the content of FlowFiles for terms that are found in a user-supplied dictionary. If a term is matched, the UTF-8 "
|
||||
+ "encoded version of the term will be added to the FlowFile using the 'matching.term' attribute")
|
||||
|
|
|
@ -26,13 +26,15 @@ import java.util.Set;
|
|||
import java.util.UUID;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
|
@ -48,6 +50,7 @@ import org.apache.nifi.processor.util.StandardValidators;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"segment", "split"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Segments a FlowFile into multiple smaller segments on byte boundaries. Each segment is given the following attributes: "
|
||||
+ "fragment.identifier, fragment.index, fragment.count, segment.original.filename; these attributes can then be used by the "
|
||||
+ "MergeContent processor in order to reconstitute the original FlowFile")
|
||||
|
|
|
@ -33,14 +33,16 @@ import java.util.concurrent.atomic.AtomicReference;
|
|||
import org.apache.commons.codec.DecoderException;
|
||||
import org.apache.commons.codec.binary.Hex;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.components.AllowableValue;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.components.ValidationContext;
|
||||
|
@ -64,6 +66,7 @@ import org.apache.nifi.util.Tuple;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"content", "split", "binary"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Splits incoming FlowFiles by a specified byte sequence")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "fragment.identifier", description = "All split FlowFiles produced from the same parent FlowFile will have the same randomly generated UUID added for this attribute"),
|
||||
|
|
|
@ -16,12 +16,21 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.InvalidJsonException;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.PathNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -38,21 +47,16 @@ import org.apache.nifi.processor.Relationship;
|
|||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import com.jayway.jsonpath.DocumentContext;
|
||||
import com.jayway.jsonpath.InvalidJsonException;
|
||||
import com.jayway.jsonpath.JsonPath;
|
||||
import com.jayway.jsonpath.PathNotFoundException;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"json", "split", "jsonpath"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Splits a JSON File into multiple, separate FlowFiles for an array element specified by a JsonPath expression. "
|
||||
+ "Each generated FlowFile is comprised of an element of the specified array and transferred to relationship 'split,' "
|
||||
+ "with the original file transferred to the 'original' relationship. If the specified JsonPath is not found or "
|
||||
|
|
|
@ -16,33 +16,6 @@
|
|||
*/
|
||||
package org.apache.nifi.processors.standard;
|
||||
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.stream.io.ByteArrayOutputStream;
|
||||
import org.apache.nifi.stream.io.ByteCountingInputStream;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.IntegerHolder;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
|
@ -53,13 +26,43 @@ import java.util.HashSet;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.UUID;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
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.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.stream.io.BufferedOutputStream;
|
||||
import org.apache.nifi.stream.io.ByteArrayOutputStream;
|
||||
import org.apache.nifi.stream.io.ByteCountingInputStream;
|
||||
import org.apache.nifi.util.IntegerHolder;
|
||||
import org.apache.nifi.util.ObjectHolder;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"split", "text"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Splits a text file into multiple smaller text files on line boundaries, each having up to a configured number of lines")
|
||||
@WritesAttributes({
|
||||
@WritesAttribute(attribute = "text.line.count", description = "The number of lines of text from the original FlowFile that were copied to this FlowFile"),
|
||||
|
|
|
@ -29,27 +29,28 @@ import javax.xml.parsers.ParserConfigurationException;
|
|||
import javax.xml.parsers.SAXParser;
|
||||
import javax.xml.parsers.SAXParserFactory;
|
||||
|
||||
import org.apache.commons.lang3.StringEscapeUtils;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
import org.apache.nifi.processor.AbstractProcessor;
|
||||
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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.io.OutputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.processors.standard.util.XmlElementNotifier;
|
||||
import org.apache.nifi.stream.io.BufferedInputStream;
|
||||
import org.apache.nifi.util.BooleanHolder;
|
||||
|
||||
import org.apache.commons.lang3.StringEscapeUtils;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import org.xml.sax.Attributes;
|
||||
|
@ -63,6 +64,7 @@ import org.xml.sax.XMLReader;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"xml", "split"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Splits an XML File into multiple separate FlowFiles, each comprising a child or descendant of the original root element")
|
||||
public class SplitXml extends AbstractProcessor {
|
||||
|
||||
|
|
|
@ -35,6 +35,8 @@ import javax.xml.transform.stream.StreamSource;
|
|||
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
|
@ -62,6 +64,7 @@ import org.apache.nifi.util.Tuple;
|
|||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@Tags({"xml", "xslt", "transform"})
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@CapabilityDescription("Applies the provided XSLT file to the flowfile XML payload. A new FlowFile is created "
|
||||
+ "with transformed content and is routed to the 'success' relationship. If the XSL transform "
|
||||
+ "fails, the original FlowFile is routed to the 'failure' relationship")
|
||||
|
|
|
@ -35,14 +35,16 @@ import org.apache.commons.compress.archivers.tar.TarArchiveEntry;
|
|||
import org.apache.commons.compress.archivers.tar.TarArchiveInputStream;
|
||||
import org.apache.commons.compress.archivers.zip.ZipArchiveInputStream;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.ReadsAttribute;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttributes;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.SeeAlso;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.flowfile.attributes.CoreAttributes;
|
||||
|
@ -67,6 +69,7 @@ import org.apache.nifi.util.ObjectHolder;
|
|||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"Unpack", "un-merge", "tar", "zip", "archive", "flowfile-stream", "flowfile-stream-v3"})
|
||||
@CapabilityDescription("Unpacks the content of FlowFiles that have been packaged with one of several different Packaging Formats, emitting one to many "
|
||||
+ "FlowFiles for each input FlowFile")
|
||||
|
|
|
@ -31,6 +31,14 @@ import javax.xml.validation.Schema;
|
|||
import javax.xml.validation.SchemaFactory;
|
||||
import javax.xml.validation.Validator;
|
||||
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
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.components.PropertyDescriptor;
|
||||
import org.apache.nifi.flowfile.FlowFile;
|
||||
import org.apache.nifi.logging.ProcessorLog;
|
||||
|
@ -39,21 +47,15 @@ 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.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.SupportsBatching;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.processor.io.InputStreamCallback;
|
||||
import org.apache.nifi.processor.util.StandardValidators;
|
||||
import org.apache.nifi.util.BooleanHolder;
|
||||
|
||||
import org.xml.sax.SAXException;
|
||||
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@SupportsBatching
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"xml", "schema", "validation", "xsd"})
|
||||
@CapabilityDescription("Validates the contents of FlowFiles against a user-specified XML Schema file")
|
||||
public class ValidateXml extends AbstractProcessor {
|
||||
|
|
|
@ -31,9 +31,13 @@ import java.util.concurrent.ConcurrentMap;
|
|||
import java.util.concurrent.atomic.AtomicReference;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.DynamicProperty;
|
||||
import org.apache.nifi.annotation.behavior.EventDriven;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement;
|
||||
import org.apache.nifi.annotation.behavior.InputRequirement.Requirement;
|
||||
import org.apache.nifi.annotation.behavior.SideEffectFree;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
import org.apache.nifi.annotation.documentation.CapabilityDescription;
|
||||
import org.apache.nifi.annotation.documentation.Tags;
|
||||
import org.apache.nifi.annotation.lifecycle.OnScheduled;
|
||||
|
@ -57,11 +61,9 @@ import org.apache.nifi.search.Searchable;
|
|||
import org.apache.nifi.update.attributes.Action;
|
||||
import org.apache.nifi.update.attributes.Condition;
|
||||
import org.apache.nifi.update.attributes.Criteria;
|
||||
import org.apache.nifi.update.attributes.Rule;
|
||||
import org.apache.nifi.update.attributes.FlowFilePolicy;
|
||||
import org.apache.nifi.update.attributes.Rule;
|
||||
import org.apache.nifi.update.attributes.serde.CriteriaSerDe;
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.nifi.annotation.behavior.WritesAttribute;
|
||||
|
||||
/**
|
||||
* This processor supports updating flowfile attributes and can do so
|
||||
|
@ -116,6 +118,7 @@ import org.apache.nifi.annotation.behavior.WritesAttribute;
|
|||
*/
|
||||
@EventDriven
|
||||
@SideEffectFree
|
||||
@InputRequirement(Requirement.INPUT_REQUIRED)
|
||||
@Tags({"attributes", "modification", "update", "delete", "Attribute Expression Language"})
|
||||
@CapabilityDescription("Updates the Attributes for a FlowFile by using the Attribute Expression Language and/or deletes the attributes based on a regular expression")
|
||||
@DynamicProperty(name = "A FlowFile attribute to update", value = "The value to set it to", supportsExpressionLanguage = true,
|
||||
|
|
Loading…
Reference in New Issue