Re: DirectRunner, Fusion, and Triggers

2021-05-19 Thread Bashir Sadjad
On Mon, May 17, 2021 at 1:07 PM Brian Hulette wrote: > > P.S. I need this pipeline to work both on a distributed runner and also > on a local machine with many cores. That's why the performance of > DirectRunner is important to me. > > IIUC the DirectRunner has intentionally made some trade-offs

Re: DirectRunner, Fusion, and Triggers

2021-05-17 Thread Brian Hulette
> P.S. I need this pipeline to work both on a distributed runner and also on a local machine with many cores. That's why the performance of DirectRunner is important to me. IIUC the DirectRunner has intentionally made some trade-offs to make it less performant, so that it better verifies pipelines

Re: DirectRunner, Fusion, and Triggers

2021-05-17 Thread Jan Lukavský
On 5/17/21 3:46 PM, Bashir Sadjad wrote: Thanks Jan. Two points: - I was running all the experiments I reported with `--targetParallelism=1` to make sure concurrent threads do not mess up the logs. I think that is what causes what you see. Try to increase the parallelism to number higher than

Re: DirectRunner, Fusion, and Triggers

2021-05-17 Thread Bashir Sadjad
Thanks Jan. Two points: - I was running all the experiments I reported with `--targetParallelism=1` to make sure concurrent threads do not mess up the logs. - I have been tracking bundles too (see @StartBundle log messages in the mini-example in my previous reply to Kenn). So I don't think bundle

Re: DirectRunner, Fusion, and Triggers

2021-05-17 Thread Jan Lukavský
Hi Bashir, the behavior you describe should be expected. DirectRunner splits the input work into bundles, processing each bundle might result in zero, one or more new bundles. The executor executes the work associated with these bundles, enqueuing new bundles into a queue, until there are no

Re: DirectRunner, Fusion, and Triggers

2021-05-12 Thread Bashir Sadjad
Thanks Kenn. On Wed, May 12, 2021 at 12:14 PM Kenneth Knowles wrote: > > On Sat, May 8, 2021 at 12:00 AM Bashir Sadjad wrote: > >> However, if I add a dummy S2' after S2 (i.e., S1->S2->S2'->S3) which only >> prints some log messages for each record and passes the record to output, >> then it se

Re: DirectRunner, Fusion, and Triggers

2021-05-12 Thread Kenneth Knowles
On Sat, May 8, 2021 at 12:00 AM Bashir Sadjad wrote: > Hi Beam-users, > > *TL;DR;* I wonder if DirectRunner does any fusion optimization > > and whether this has any impact on triggers/panes? > > *Details* (the context for

DirectRunner, Fusion, and Triggers

2021-05-07 Thread Bashir Sadjad
Hi Beam-users, *TL;DR;* I wonder if DirectRunner does any fusion optimization and whether this has any impact on triggers/panes? *Details* (the context for everything below is *DirectRunner* and this is a *batch* job): I hav