> Are we seeing similar discrepancies for Flink?

I am not sure I'm afraid (no easy access to flink right now). I tried
without success to get Apex runner going on Cloudera YARN for this today -
I'll keep trying when time allows.

I've updated the DAGs to show more detail:
https://github.com/gbif/beam-perf/tree/master/avro-to-avro

On Wed, Aug 22, 2018 at 1:41 PM Robert Bradshaw <[email protected]> wrote:

> That is quite the DAG... Are we seeing similar discrepancies for
> Flink? (Trying to understand if this is Beam->Spark translation bloat,
> or inherent to the WriteFiles transform itself.)
> On Wed, Aug 22, 2018 at 1:35 PM Tim Robertson <[email protected]>
> wrote:
> >
> > Thanks Robert
> >
> > > Have you profiled to see which stages and/or operations are taking up
> the time?
> >
> > Not yet. I'm browsing through the spark DAG produced which I've
> committed [1] and reading the code.
> >
> > [1] https://github.com/gbif/beam-perf/tree/master/avro-to-avro
> >
> > On Wed, Aug 22, 2018 at 12:12 PM Robert Bradshaw <[email protected]>
> wrote:
> >>
> >> I agree that this is concerning. Some of the complexity may have also
> >> been introduced to accommodate writing files in Streaming mode, but it
> >> seems we should be able to execute this as a single Map operation.
> >>
> >> Have you profiled to see which stages and/or operations are taking up
> the time?
> >> On Wed, Aug 22, 2018 at 11:29 AM Tim Robertson
> >> <[email protected]> wrote:
> >> >
> >> > Hi folks,
> >> >
> >> > I've recently been involved in projects rewriting Avro files and have
> discovered a concerning performance trait in Beam.
> >> >
> >> > I have observed Beam between 6-20x slower than native Spark or
> MapReduce code for a simple pipeline of read Avro, modify, write Avro.
> >> >
> >> >  - Rewriting 200TB of Avro files (big cluster): 14 hrs using
> Beam/Spark, 40 minutes with a map-only MR job
> >> >  - Rewriting 1.5TB Avro file (small cluster): 2 hrs using Beam/Spark,
> 18 minutes using vanilla Spark code. Test code available [1]
> >> >
> >> > These tests were running Beam 2.6.0 on Cloudera 5.12.x clusters
> (Spark / YARN) on reference Dell / Cloudera hardware.
> >> >
> >> > I have only just started exploring but I believe the cause is rooted
> in the WriteFiles which is used by all our file based IO. WriteFiles is
> reasonably complex with reshuffles, spilling to temporary files (presumably
> to accommodate varying bundle sizes/avoid small files), a union, a GBK etc.
> >> >
> >> > Before I go too far with exploration I'd appreciate thoughts on
> whether we believe this is a concern (I do), if we should explore
> optimisations or any insight from previous work in this area.
> >> >
> >> > Thanks,
> >> > Tim
> >> >
> >> > [1] https://github.com/gbif/beam-perf/tree/master/avro-to-avro
>

Reply via email to