git.net

[Date Prev][Date Next][Thread Prev][Thread Next][Date Index][Thread Index]

Re: Advice on parallelizing network calls in DoFn


I will start with the "exciting futuristic" answer, which is that we envision the new DoFn to be able to provide an automatic ExecutorService parameters that you can use as you wish.

    new DoFn<>() {
      @ProcessElement
      public void process(ProcessContext ctx, ExecutorService executorService) {
          ... launch some futures, put them in instance vars ...
      }

      @FinishBundle
      public void finish(...) {
         ... block on futures, output results if appropriate ...
      }
    }

This way, the Java SDK harness can use its overarching knowledge of what is going on in a computation to, for example, share a thread pool between different bits. This was one reason to delete IntraBundleParallelization - it didn't allow the runner and user code to properly manage how many things were going on concurrently. And mostly the runner should own parallelizing to max out cores and what user code needs is asynchrony hooks that can interact with that. However, this feature is not thoroughly considered. TBD how much the harness itself manages blocking on outstanding requests versus it being your responsibility in FinishBundle, etc.

I haven't explored rolling your own here, if you are willing to do the knob tuning to get the threading acceptable for your particular use case. Perhaps someone else can weigh in.

Kenn

On Fri, Mar 9, 2018 at 1:38 PM Josh Ferge <josh.ferge@xxxxxxxxxxxxxxxxxx> wrote:
Hello all:

Our team has a pipeline that make external network calls. These pipelines are currently super slow, and the hypothesis is that they are slow because we are not threading for our network calls. The github issue below provides some discussion around this:


In beam 1.0, there was IntraBundleParallelization, which helped with this. However, this was removed because it didn't comply with a few BEAM paradigms. 

Questions going forward:

What is advised for jobs that make blocking network calls? It seems bundling the elements into groups of size X prior to passing to the DoFn, and managing the threading within the function might work. thoughts?
Are these types of jobs even suitable for beam?
Are there any plans to develop features that help with this?

Thanks


( ! ) Warning: include(msgfooter.php): failed to open stream: No such file or directory in /var/www/git/apache-beam-users/msg02035.html on line 70
Call Stack
#TimeMemoryFunctionLocation
10.0009372792{main}( ).../msg02035.html:0

( ! ) Warning: include(): Failed opening 'msgfooter.php' for inclusion (include_path='.:/var/www/git') in /var/www/git/apache-beam-users/msg02035.html on line 70
Call Stack
#TimeMemoryFunctionLocation
10.0009372792{main}( ).../msg02035.html:0