Hi Peter,

1) You can achieve what you described by creating the output flowfiles up
front, then doing a single InputStreamCallback, and inside the callback
make calls to session.append(FlowFile flowFile, OutputStreamCallback
writer) for each output flowfile you want to write to.
Behind the scenes the framework will manage the output streams so that each
call to append doesn't have to open a new output stream.

2) Yes calling session.commit() will release any transferred flow files,
most likely something you want to do if a single call to onTrigger might be
producing a ton of flow files. You essentially have a brand new session
after calling commit().

Another concept I wanted to mention, sort of related to your commit
question, is the @SupportsBatching annotation. If a processor has this
annotation, there is a slider in the processor UI where you can configure a
time threshold, say 100ms.
This lets the framework take 100ms worth of processor activity, and batch
it together so that there only has to be one real session.commit(). It lets
the user choose between latency and throughput...
If they want low latency they keep it at 0ms, if they want to sacrifice
some latency for higher throughput they can drag the slider to the right.

-Bryan


On Tue, Aug 23, 2016 at 1:17 PM, Peter Wicks (pwicks) <[email protected]>
wrote:

> I have a couple of innards questions:
>
>
> 1.       I'm familiar with how to write flow files, but what if I want to
> write to multiple FlowFiles in parallel, rather than serially? The use case
> I have in mind involves reading through an input stream and generating
> multiple derivative copies in different formats without ever reading the
> original stream more than once.  I tried capturing the OutputStream
> variable from the callback, but ends up getting closed sometimes before I
> am done writing.
>
> flowFile = session.write(flowFile, new StreamCallback() {
>                 @Override
>                 public void process(final InputStream rawIn, final
> OutputStream rawOut) throws IOException {
> }
> });
>
>
> 2.       If I explicitly call ` session.commit();` does this cause
> transferred FlowFiles to be immediately released to their appropriate
> queues?  If yes, can I continue to work with the session object after
> commit and release batches to the queue? Think of it like a SQL dirty read,
> `FlowFile Dirty Read`.
>
> Regards,
>   Peter
>

Reply via email to