Developer Guide
Also available as:
PDF
loading table of contents...

When Processors are Triggered

A Processor's onTrigger method will be called only when it is scheduled to run and when work exists for the Processor. Work is said to exist for a Processor if any of the following conditions is met:

  • A Connection whose destination is the Processor has at least one FlowFile in its queue

  • The Processors has no incoming Connections

  • The Processor is annotated with the @TriggerWhenEmpty annotation

Several factors exist that will contribute to when a Processor's onTrigger method is invoked. First, the Processor will not be triggered unless a user has configured the Processor to run. If a Processor is scheduled to run, the Framework periodically (the period is configured by users in the User Interface) checks if there is work for the Processor to do, as described above. If so, the Framework will check downstream destinations of the Processor. If any of the Processor's outbound Connections is full, by default, the Processor will not be scheduled to run.

However, the @TriggerWhenAnyDestinationAvailable annotation may be added to the Processor's class. In this case, the requirement is changed so that only one downstream destination must be "available" (a destination is considered "available" if the Connection's queue is not full), rather than requiring that all downstream destinations be available.

Also related to Processor scheduling is the @TriggerSerially annotation. Processors that use this Annotation will never have more than one thread running the onTrigger method simultaneously. It is crucial to note, though, that the thread executing the code may change from invocation to invocation. Therefore, care must still be taken to ensure that the Processor is thread-safe!