By the way, you can take a look on JdbcIO which does a reshuffle transform to 
avoid the "fusion" issue.

Regards
JB

Le 15 mars 2018 à 10:44, à 10:44, Raghu Angadi <[email protected]> a écrit:
>In streaming, a simple way is to add a reshuffle to increase
>parallelism.
>When you are external-call bound, extra cost of reshuffle is
>negligible.
>e.g.
>https://stackoverflow.com/questions/46116443/dataflow-streaming-job-not-scaleing-past-1-worker
>
>Note that by default Dataflow workers use a couple of hundred threads
>as
>required. This can be increased with a pipeline option if you prefer. I
>am
>not sure of other runners.
>
>On Thu, Mar 15, 2018 at 8:25 AM Falcon Taylor-Carter <
>[email protected]> wrote:
>
>> Hello Pablo,
>>
>> Thanks for checking up (I'm working with Josh on this problem). It
>seems
>> there isn't a built-in process for this kind of use case currently,
>and
>> that the best process right now is to handle our own bundling and
>threading
>> in the DoFn. If you had any other suggestions, or anything to keep in
>mind
>> in doing this, let us know!
>>
>> Falcon
>>
>> On Tue, Mar 13, 2018 at 4:52 PM, Pablo Estrada <[email protected]>
>wrote:
>>
>>> I'd just like to close the loop. Josh, did you get an
>answer/guidance on
>>> how to proceed with your pipeline?
>>> Or maybe we'll need a new thread to figure that out : )
>>> Best
>>> -P.
>>>
>>>
>>> On Fri, Mar 9, 2018 at 1:39 PM Josh Ferge
><[email protected]>
>>> 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:
>>>>
>>>> https://github.com/apache/beam/pull/957
>>>>
>>>> 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
>>>>
>>> --
>>> Got feedback? go/pabloem-feedback
>>> <https://goto.google.com/pabloem-feedback>
>>>
>>
>>

Reply via email to