On Wed, Jul 20, 2022 at 12:57 PM Chamikara Jayalath <[email protected]>
wrote:

> I don't think it's an antipattern per se. You can implement arbitrary
> operations in a DoFn or an SDF to read data.
>
> But if a single resource ID maps to a large amount of data, Beam runners
> (including Dataflow) will be able to parallelize reading, hence your
> solution may have suboptimal performance compared to reading from a Beam
> source that can be fully parallelized.
>

*will not be able to*


>
> Thanks,
> Cham
>
> On Wed, Jul 20, 2022 at 11:53 AM Shree Tanna <[email protected]>
> wrote:
>
>> Thank you!
>> I will try this out.
>> One more question on this, is it considered anti-pattern to do HTTP
>> ingestion on GCP Dataflow due to the reasoning I mentioned in my original
>> message? I ask because I am getting that indication from some of my
>> co-workers and also from google cloud support. Not sure if this is the
>> right place to ask this question. Happy to move this conversation to
>> somewhere else if not.
>>
>> On Tue, Jul 19, 2022 at 5:18 PM Luke Cwik via user <[email protected]>
>> wrote:
>>
>>> Even if you don't have the resource ids ahead of time, you can have a
>>> pipeline like:
>>> Impulse -> ParDo(GenerateResourceIds) -> Reshuffle ->
>>> ParDo(ReadResourceIds) -> ...
>>>
>>> You could also compose these as splittable DoFns [1, 2, 3]:
>>> ParDo(SplittableGenerateResourceIds) -> ParDo(SplittableReadResourceIds)
>>>
>>> The first approach is the simplest as the reshuffle will rebalance the
>>> reading of each resource id across worker nodes but is limited in
>>> generating resource ids on one worker. Making the generation a splittable
>>> DoFn will mean that you can increase the parallelism of generation which is
>>> important if there are so many that it could crash a worker or fail to have
>>> the output committed (these kinds of failures are runner dependent on how
>>> well they handle single bundles with large outputs). Making the reading
>>> splittable allows you to handle a large resource (imagine a large file) so
>>> that it can be read and processed in parallel (and will have similar
>>> failures if the runner can't handle single bundles with large outputs).
>>>
>>> You can always start with the first solution and swap either piece to be
>>> a splittable DoFn depending on your performance requirements and how well
>>> the simple solution works.
>>>
>>> 1: https://beam.apache.org/blog/splittable-do-fn/
>>> 2: https://beam.apache.org/blog/splittable-do-fn-is-available/
>>> 3:
>>> https://beam.apache.org/documentation/programming-guide/#splittable-dofns
>>>
>>>
>>> On Tue, Jul 19, 2022 at 10:05 AM Damian Akpan <[email protected]>
>>> wrote:
>>>
>>>> Provided you have all the resources ids ahead of fetching, Beam will
>>>> spread the fetches to its workers. It will still fetch synchronously but
>>>> within that worker.
>>>>
>>>> On Tue, Jul 19, 2022 at 5:40 PM Shree Tanna <[email protected]>
>>>> wrote:
>>>>
>>>>> Hi all,
>>>>>
>>>>> I'm planning to use Apache beam to extract and load part of the ETL
>>>>> pipeline and run the jobs on Dataflow. I will have to do the REST API
>>>>> ingestion on our platform. I can opt to make sync API calls from DoFn. 
>>>>> With
>>>>> that pipelines will stall while REST requests are made over the network.
>>>>>
>>>>> Is it best practice to run the REST ingestion job on Dataflow? Is
>>>>> there any best practice I can follow to accomplish this? Just as a
>>>>> reference I'm adding this
>>>>> <https://stackoverflow.com/questions/50335521/best-practices-in-http-calls-in-cloud-dataflow-java>
>>>>> StackOverflow thread here too. Also, I notice that Rest I/O transform
>>>>> <https://beam.apache.org/documentation/io/built-in/> built-in
>>>>> connector is in progress for Java.
>>>>>
>>>>> Let me know if this is the right group to ask this question. I can
>>>>> also ask [email protected] if needed.
>>>>> --
>>>>> Thanks,
>>>>> Shree
>>>>>
>>>>
>>
>> --
>> Best,
>> Shree
>>
>

Reply via email to