You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Jiadai Xia <da...@google.com> on 2020/08/21 17:54:05 UTC

Splittable-Dofn not distributing the work to multiple workers

Hi,
As stated in the title, I tried to implement a SDF for reading the Parquet
file and I am trying to run it with Dataflow runner. As the initial split
outputs a bunch of ranges but the number of workers are not scaled up and
the work is not distributed. Any suggestion on what can be the problem?
I have tested it with Direct runner and the parallelism looks fine on small
samples on Direct Runner.
Below is my implementation of the SDF
https://github.com/apache/beam/pull/12223
-- 





*Jiadai Xia*

SWE Intern

1 (646) 413 8071

danielxjd@google.com

<https://www.linkedin.com/company/google/>
<https://www.youtube.com/user/lifeatgoogle>
<https://www.facebook.com/lifeatgoogle/> <https://twitter.com/lifeatgoogle>

<https://www.instagram.com/lifeatgoogle>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Chamikara Jayalath <ch...@google.com>.
Only explanation I can think of is a hotkey where Dataflow would quickly go
through most of the elements of the GBK (that follows initial splitting)
and will be stuck on one/few elements without being able to scale. But it
seems like Jiadai has ruled this out.

On Fri, Aug 21, 2020 at 4:02 PM Jiadai Xia <da...@google.com> wrote:

> I have also tried to run with auto scaling on, so the split restriction
> generates 1000 element and the number of worker goes from 1 to 2 for a few
> minutes than it change back to 1 worker. I have also printed out some log
> info to show that for each range the amount of work is the same and there
> should not be a hot key element. But the dataflow does not scale up.
>
> On Fri, Aug 21, 2020 at 3:50 PM Boyuan Zhang <bo...@google.com> wrote:
>
>> Jiadai discussed this problem with me before. At that moment, I was
>> suspecting that the num of (element, restriction) pair is too small for
>> Reshuffle to distribute them among many workers. Also since your job is
>> started with 3 workers, it's possible that 3 workers are sufficient for all
>> work.
>>
>> On Fri, Aug 21, 2020 at 3:26 PM Luke Cwik <lc...@google.com> wrote:
>>
>>> For some reason I thought we were talking about Python. Cham is correct
>>> that there is no FnAPI so there is no buffer that is in the way so what I
>>> said doesn't apply. So the reason for no splitting is a mystery to me, is
>>> it possible that you haven't waited long enough?
>>>
>>> On Fri, Aug 21, 2020 at 3:11 PM Chamikara Jayalath <ch...@google.com>
>>> wrote:
>>>
>>>>
>>>>
>>>> On Fri, Aug 21, 2020 at 2:03 PM Luke Cwik <lc...@google.com> wrote:
>>>>
>>>>> The problem is that the GBK has so little data it is being placed into
>>>>> a single work item and all this data ends up in a network buffer between
>>>>> the Runner and the SDK so splitting the GBK output never happens.
>>>>>
>>>>
>>>> There's no Fn API boundary for runner v1. So I'm not sure why buffered
>>>> splitting will be an issue for initial splitting for SDFs on runner v1. Or
>>>> did you mean the boundary between Dataflow service and Dataflow worker+SDK.
>>>> In this case, It could be that the amount of elements in the GBK was too
>>>> small for Dataflow to register a split for the GBK (initial or dynamic).
>>>>
>>>>
>>>>>
>>>>> You'll need to use runner v2 to get around this issue since it is the
>>>>> only one that solves the buffered splitting problem. Runner v1 only
>>>>> supports splitting the "unread" portion of the GBK while runner v2 supports
>>>>> splitting everything from the current element being processed including
>>>>> what has been buffered after it in addition to the "unread" portion of the
>>>>> GBK.
>>>>>
>>>>
>>>> I don't think trying out v2 is an option for this since runner v2 is
>>>> only available for Python pipelines.
>>>>
>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com>
>>>>> wrote:
>>>>>
>>>>>> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
>>>>>> When I check into the reshuffle before the processing, it seems that
>>>>>> the GroupbyKey function output one element each time and wait until the
>>>>>> processing part finish reading the element and then it start to output the
>>>>>> next element.
>>>>>> Is that the correct order of operation or is it altered by the
>>>>>> dataflow fusion? I have set the num of workers to be 3 and disabled the
>>>>>> autoscaling.
>>>>>>
>>>>>> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>>>>>>
>>>>>>> Yes it does.
>>>>>>>
>>>>>>> There should be a reshuffle between the initial splitting and the
>>>>>>> processing portion.
>>>>>>>
>>>>>>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> I am using v1. Does v1 support the initial splitting and
>>>>>>>> distribution? since I expect it to distribute the initial splitting to
>>>>>>>> multiple workers.
>>>>>>>>
>>>>>>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Are you using Dataflow runner v2[1] since the default for Beam
>>>>>>>>> Java still uses Dataflow runner v1?
>>>>>>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>>>>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>>>>>>
>>>>>>>>> 1:
>>>>>>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>>>>>>
>>>>>>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>>>>>>> wrote:
>>>>>>>>>
>>>>>>>>>> Hi,
>>>>>>>>>> As stated in the title, I tried to implement a SDF for reading
>>>>>>>>>> the Parquet file and I am trying to run it with Dataflow runner. As the
>>>>>>>>>> initial split outputs a bunch of ranges but the number of workers are not
>>>>>>>>>> scaled up and the work is not distributed. Any suggestion on what can be
>>>>>>>>>> the problem?
>>>>>>>>>> I have tested it with Direct runner and the parallelism looks
>>>>>>>>>> fine on small samples on Direct Runner.
>>>>>>>>>> Below is my implementation of the SDF
>>>>>>>>>> https://github.com/apache/beam/pull/12223
>>>>>>>>>> --
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>> *Jiadai Xia*
>>>>>>>>>>
>>>>>>>>>> SWE Intern
>>>>>>>>>>
>>>>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>>>>
>>>>>>>>>> danielxjd@google.com
>>>>>>>>>>
>>>>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>>>>
>>>>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>>>
>>>>>>>>
>>>>>>>> --
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> *Jiadai Xia*
>>>>>>>>
>>>>>>>> SWE Intern
>>>>>>>>
>>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>>
>>>>>>>> danielxjd@google.com
>>>>>>>>
>>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>>
>>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>> --
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> *Jiadai Xia*
>>>>>>
>>>>>> SWE Intern
>>>>>>
>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>
>>>>>> danielxjd@google.com
>>>>>>
>>>>>> <https://www.linkedin.com/company/google/>
>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>
>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>
>>>>>>
>>>>>>
>
> --
>
>
>
>
>
> *Jiadai Xia*
>
> SWE Intern
>
> 1 (646) 413 8071 <(646)%20413-8071>
>
> danielxjd@google.com
>
> <https://www.linkedin.com/company/google/>
> <https://www.youtube.com/user/lifeatgoogle>
> <https://www.facebook.com/lifeatgoogle/>
> <https://twitter.com/lifeatgoogle>
>
> <https://www.instagram.com/lifeatgoogle>
>
>
>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Jiadai Xia <da...@google.com>.
I have also tried to run with auto scaling on, so the split restriction
generates 1000 element and the number of worker goes from 1 to 2 for a few
minutes than it change back to 1 worker. I have also printed out some log
info to show that for each range the amount of work is the same and there
should not be a hot key element. But the dataflow does not scale up.

On Fri, Aug 21, 2020 at 3:50 PM Boyuan Zhang <bo...@google.com> wrote:

> Jiadai discussed this problem with me before. At that moment, I was
> suspecting that the num of (element, restriction) pair is too small for
> Reshuffle to distribute them among many workers. Also since your job is
> started with 3 workers, it's possible that 3 workers are sufficient for all
> work.
>
> On Fri, Aug 21, 2020 at 3:26 PM Luke Cwik <lc...@google.com> wrote:
>
>> For some reason I thought we were talking about Python. Cham is correct
>> that there is no FnAPI so there is no buffer that is in the way so what I
>> said doesn't apply. So the reason for no splitting is a mystery to me, is
>> it possible that you haven't waited long enough?
>>
>> On Fri, Aug 21, 2020 at 3:11 PM Chamikara Jayalath <ch...@google.com>
>> wrote:
>>
>>>
>>>
>>> On Fri, Aug 21, 2020 at 2:03 PM Luke Cwik <lc...@google.com> wrote:
>>>
>>>> The problem is that the GBK has so little data it is being placed into
>>>> a single work item and all this data ends up in a network buffer between
>>>> the Runner and the SDK so splitting the GBK output never happens.
>>>>
>>>
>>> There's no Fn API boundary for runner v1. So I'm not sure why buffered
>>> splitting will be an issue for initial splitting for SDFs on runner v1. Or
>>> did you mean the boundary between Dataflow service and Dataflow worker+SDK.
>>> In this case, It could be that the amount of elements in the GBK was too
>>> small for Dataflow to register a split for the GBK (initial or dynamic).
>>>
>>>
>>>>
>>>> You'll need to use runner v2 to get around this issue since it is the
>>>> only one that solves the buffered splitting problem. Runner v1 only
>>>> supports splitting the "unread" portion of the GBK while runner v2 supports
>>>> splitting everything from the current element being processed including
>>>> what has been buffered after it in addition to the "unread" portion of the
>>>> GBK.
>>>>
>>>
>>> I don't think trying out v2 is an option for this since runner v2 is
>>> only available for Python pipelines.
>>>
>>>
>>>>
>>>>
>>>>
>>>> On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com>
>>>> wrote:
>>>>
>>>>> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
>>>>> When I check into the reshuffle before the processing, it seems that
>>>>> the GroupbyKey function output one element each time and wait until the
>>>>> processing part finish reading the element and then it start to output the
>>>>> next element.
>>>>> Is that the correct order of operation or is it altered by the
>>>>> dataflow fusion? I have set the num of workers to be 3 and disabled the
>>>>> autoscaling.
>>>>>
>>>>> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>>>>>
>>>>>> Yes it does.
>>>>>>
>>>>>> There should be a reshuffle between the initial splitting and the
>>>>>> processing portion.
>>>>>>
>>>>>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I am using v1. Does v1 support the initial splitting and
>>>>>>> distribution? since I expect it to distribute the initial splitting to
>>>>>>> multiple workers.
>>>>>>>
>>>>>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>>>>>>
>>>>>>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>>>>>>> still uses Dataflow runner v1?
>>>>>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>>>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>>>>>
>>>>>>>> 1:
>>>>>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>>>>>
>>>>>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>>>>>> wrote:
>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>> As stated in the title, I tried to implement a SDF for reading the
>>>>>>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>>>>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>>>>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>>>>>>> I have tested it with Direct runner and the parallelism looks fine
>>>>>>>>> on small samples on Direct Runner.
>>>>>>>>> Below is my implementation of the SDF
>>>>>>>>> https://github.com/apache/beam/pull/12223
>>>>>>>>> --
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> *Jiadai Xia*
>>>>>>>>>
>>>>>>>>> SWE Intern
>>>>>>>>>
>>>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>>>
>>>>>>>>> danielxjd@google.com
>>>>>>>>>
>>>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>>>
>>>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> *Jiadai Xia*
>>>>>>>
>>>>>>> SWE Intern
>>>>>>>
>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>
>>>>>>> danielxjd@google.com
>>>>>>>
>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>
>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>> --
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> *Jiadai Xia*
>>>>>
>>>>> SWE Intern
>>>>>
>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>
>>>>> danielxjd@google.com
>>>>>
>>>>> <https://www.linkedin.com/company/google/>
>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>> <https://twitter.com/lifeatgoogle>
>>>>>
>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>
>>>>>
>>>>>

-- 





*Jiadai Xia*

SWE Intern

1 (646) 413 8071

danielxjd@google.com

<https://www.linkedin.com/company/google/>
<https://www.youtube.com/user/lifeatgoogle>
<https://www.facebook.com/lifeatgoogle/> <https://twitter.com/lifeatgoogle>

<https://www.instagram.com/lifeatgoogle>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Boyuan Zhang <bo...@google.com>.
Jiadai discussed this problem with me before. At that moment, I was
suspecting that the num of (element, restriction) pair is too small for
Reshuffle to distribute them among many workers. Also since your job is
started with 3 workers, it's possible that 3 workers are sufficient for all
work.

On Fri, Aug 21, 2020 at 3:26 PM Luke Cwik <lc...@google.com> wrote:

> For some reason I thought we were talking about Python. Cham is correct
> that there is no FnAPI so there is no buffer that is in the way so what I
> said doesn't apply. So the reason for no splitting is a mystery to me, is
> it possible that you haven't waited long enough?
>
> On Fri, Aug 21, 2020 at 3:11 PM Chamikara Jayalath <ch...@google.com>
> wrote:
>
>>
>>
>> On Fri, Aug 21, 2020 at 2:03 PM Luke Cwik <lc...@google.com> wrote:
>>
>>> The problem is that the GBK has so little data it is being placed into a
>>> single work item and all this data ends up in a network buffer between the
>>> Runner and the SDK so splitting the GBK output never happens.
>>>
>>
>> There's no Fn API boundary for runner v1. So I'm not sure why buffered
>> splitting will be an issue for initial splitting for SDFs on runner v1. Or
>> did you mean the boundary between Dataflow service and Dataflow worker+SDK.
>> In this case, It could be that the amount of elements in the GBK was too
>> small for Dataflow to register a split for the GBK (initial or dynamic).
>>
>>
>>>
>>> You'll need to use runner v2 to get around this issue since it is the
>>> only one that solves the buffered splitting problem. Runner v1 only
>>> supports splitting the "unread" portion of the GBK while runner v2 supports
>>> splitting everything from the current element being processed including
>>> what has been buffered after it in addition to the "unread" portion of the
>>> GBK.
>>>
>>
>> I don't think trying out v2 is an option for this since runner v2 is only
>> available for Python pipelines.
>>
>>
>>>
>>>
>>>
>>> On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com> wrote:
>>>
>>>> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
>>>> When I check into the reshuffle before the processing, it seems that
>>>> the GroupbyKey function output one element each time and wait until the
>>>> processing part finish reading the element and then it start to output the
>>>> next element.
>>>> Is that the correct order of operation or is it altered by the dataflow
>>>> fusion? I have set the num of workers to be 3 and disabled the autoscaling.
>>>>
>>>> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>>>>
>>>>> Yes it does.
>>>>>
>>>>> There should be a reshuffle between the initial splitting and the
>>>>> processing portion.
>>>>>
>>>>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com>
>>>>> wrote:
>>>>>
>>>>>> I am using v1. Does v1 support the initial splitting and
>>>>>> distribution? since I expect it to distribute the initial splitting to
>>>>>> multiple workers.
>>>>>>
>>>>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>>>>>
>>>>>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>>>>>> still uses Dataflow runner v1?
>>>>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>>>>
>>>>>>> 1:
>>>>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>>>>
>>>>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi,
>>>>>>>> As stated in the title, I tried to implement a SDF for reading the
>>>>>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>>>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>>>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>>>>>> I have tested it with Direct runner and the parallelism looks fine
>>>>>>>> on small samples on Direct Runner.
>>>>>>>> Below is my implementation of the SDF
>>>>>>>> https://github.com/apache/beam/pull/12223
>>>>>>>> --
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>> *Jiadai Xia*
>>>>>>>>
>>>>>>>> SWE Intern
>>>>>>>>
>>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>>
>>>>>>>> danielxjd@google.com
>>>>>>>>
>>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>>
>>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>
>>>>>> --
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> *Jiadai Xia*
>>>>>>
>>>>>> SWE Intern
>>>>>>
>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>
>>>>>> danielxjd@google.com
>>>>>>
>>>>>> <https://www.linkedin.com/company/google/>
>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>
>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>> --
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> *Jiadai Xia*
>>>>
>>>> SWE Intern
>>>>
>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>
>>>> danielxjd@google.com
>>>>
>>>> <https://www.linkedin.com/company/google/>
>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>> <https://www.facebook.com/lifeatgoogle/>
>>>> <https://twitter.com/lifeatgoogle>
>>>>
>>>> <https://www.instagram.com/lifeatgoogle>
>>>>
>>>>
>>>>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Luke Cwik <lc...@google.com>.
For some reason I thought we were talking about Python. Cham is correct
that there is no FnAPI so there is no buffer that is in the way so what I
said doesn't apply. So the reason for no splitting is a mystery to me, is
it possible that you haven't waited long enough?

On Fri, Aug 21, 2020 at 3:11 PM Chamikara Jayalath <ch...@google.com>
wrote:

>
>
> On Fri, Aug 21, 2020 at 2:03 PM Luke Cwik <lc...@google.com> wrote:
>
>> The problem is that the GBK has so little data it is being placed into a
>> single work item and all this data ends up in a network buffer between the
>> Runner and the SDK so splitting the GBK output never happens.
>>
>
> There's no Fn API boundary for runner v1. So I'm not sure why buffered
> splitting will be an issue for initial splitting for SDFs on runner v1. Or
> did you mean the boundary between Dataflow service and Dataflow worker+SDK.
> In this case, It could be that the amount of elements in the GBK was too
> small for Dataflow to register a split for the GBK (initial or dynamic).
>
>
>>
>> You'll need to use runner v2 to get around this issue since it is the
>> only one that solves the buffered splitting problem. Runner v1 only
>> supports splitting the "unread" portion of the GBK while runner v2 supports
>> splitting everything from the current element being processed including
>> what has been buffered after it in addition to the "unread" portion of the
>> GBK.
>>
>
> I don't think trying out v2 is an option for this since runner v2 is only
> available for Python pipelines.
>
>
>>
>>
>>
>> On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com> wrote:
>>
>>> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
>>> When I check into the reshuffle before the processing, it seems that the
>>> GroupbyKey function output one element each time and wait until the
>>> processing part finish reading the element and then it start to output the
>>> next element.
>>> Is that the correct order of operation or is it altered by the dataflow
>>> fusion? I have set the num of workers to be 3 and disabled the autoscaling.
>>>
>>> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>>>
>>>> Yes it does.
>>>>
>>>> There should be a reshuffle between the initial splitting and the
>>>> processing portion.
>>>>
>>>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com>
>>>> wrote:
>>>>
>>>>> I am using v1. Does v1 support the initial splitting and distribution?
>>>>> since I expect it to distribute the initial splitting to multiple workers.
>>>>>
>>>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>>>>
>>>>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>>>>> still uses Dataflow runner v1?
>>>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>>>
>>>>>> 1:
>>>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>>>
>>>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>> As stated in the title, I tried to implement a SDF for reading the
>>>>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>>>>> I have tested it with Direct runner and the parallelism looks fine
>>>>>>> on small samples on Direct Runner.
>>>>>>> Below is my implementation of the SDF
>>>>>>> https://github.com/apache/beam/pull/12223
>>>>>>> --
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> *Jiadai Xia*
>>>>>>>
>>>>>>> SWE Intern
>>>>>>>
>>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>>
>>>>>>> danielxjd@google.com
>>>>>>>
>>>>>>> <https://www.linkedin.com/company/google/>
>>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>>
>>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>
>>>>> --
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> *Jiadai Xia*
>>>>>
>>>>> SWE Intern
>>>>>
>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>
>>>>> danielxjd@google.com
>>>>>
>>>>> <https://www.linkedin.com/company/google/>
>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>> <https://twitter.com/lifeatgoogle>
>>>>>
>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>
>>>>>
>>>>>
>>>
>>> --
>>>
>>>
>>>
>>>
>>>
>>> *Jiadai Xia*
>>>
>>> SWE Intern
>>>
>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>
>>> danielxjd@google.com
>>>
>>> <https://www.linkedin.com/company/google/>
>>> <https://www.youtube.com/user/lifeatgoogle>
>>> <https://www.facebook.com/lifeatgoogle/>
>>> <https://twitter.com/lifeatgoogle>
>>>
>>> <https://www.instagram.com/lifeatgoogle>
>>>
>>>
>>>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Chamikara Jayalath <ch...@google.com>.
On Fri, Aug 21, 2020 at 2:03 PM Luke Cwik <lc...@google.com> wrote:

> The problem is that the GBK has so little data it is being placed into a
> single work item and all this data ends up in a network buffer between the
> Runner and the SDK so splitting the GBK output never happens.
>

There's no Fn API boundary for runner v1. So I'm not sure why buffered
splitting will be an issue for initial splitting for SDFs on runner v1. Or
did you mean the boundary between Dataflow service and Dataflow worker+SDK.
In this case, It could be that the amount of elements in the GBK was too
small for Dataflow to register a split for the GBK (initial or dynamic).


>
> You'll need to use runner v2 to get around this issue since it is the only
> one that solves the buffered splitting problem. Runner v1 only supports
> splitting the "unread" portion of the GBK while runner v2 supports
> splitting everything from the current element being processed including
> what has been buffered after it in addition to the "unread" portion of the
> GBK.
>

I don't think trying out v2 is an option for this since runner v2 is only
available for Python pipelines.


>
>
>
> On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com> wrote:
>
>> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
>> When I check into the reshuffle before the processing, it seems that the
>> GroupbyKey function output one element each time and wait until the
>> processing part finish reading the element and then it start to output the
>> next element.
>> Is that the correct order of operation or is it altered by the dataflow
>> fusion? I have set the num of workers to be 3 and disabled the autoscaling.
>>
>> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>>
>>> Yes it does.
>>>
>>> There should be a reshuffle between the initial splitting and the
>>> processing portion.
>>>
>>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com>
>>> wrote:
>>>
>>>> I am using v1. Does v1 support the initial splitting and distribution?
>>>> since I expect it to distribute the initial splitting to multiple workers.
>>>>
>>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>>>
>>>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>>>> still uses Dataflow runner v1?
>>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>>
>>>>> 1:
>>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>>
>>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>>> wrote:
>>>>>
>>>>>> Hi,
>>>>>> As stated in the title, I tried to implement a SDF for reading the
>>>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>>>> I have tested it with Direct runner and the parallelism looks fine on
>>>>>> small samples on Direct Runner.
>>>>>> Below is my implementation of the SDF
>>>>>> https://github.com/apache/beam/pull/12223
>>>>>> --
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>>
>>>>>> *Jiadai Xia*
>>>>>>
>>>>>> SWE Intern
>>>>>>
>>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>>
>>>>>> danielxjd@google.com
>>>>>>
>>>>>> <https://www.linkedin.com/company/google/>
>>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>>> <https://twitter.com/lifeatgoogle>
>>>>>>
>>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>>
>>>>>>
>>>>>>
>>>>
>>>> --
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> *Jiadai Xia*
>>>>
>>>> SWE Intern
>>>>
>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>
>>>> danielxjd@google.com
>>>>
>>>> <https://www.linkedin.com/company/google/>
>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>> <https://www.facebook.com/lifeatgoogle/>
>>>> <https://twitter.com/lifeatgoogle>
>>>>
>>>> <https://www.instagram.com/lifeatgoogle>
>>>>
>>>>
>>>>
>>
>> --
>>
>>
>>
>>
>>
>> *Jiadai Xia*
>>
>> SWE Intern
>>
>> 1 (646) 413 8071 <(646)%20413-8071>
>>
>> danielxjd@google.com
>>
>> <https://www.linkedin.com/company/google/>
>> <https://www.youtube.com/user/lifeatgoogle>
>> <https://www.facebook.com/lifeatgoogle/>
>> <https://twitter.com/lifeatgoogle>
>>
>> <https://www.instagram.com/lifeatgoogle>
>>
>>
>>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Luke Cwik <lc...@google.com>.
The problem is that the GBK has so little data it is being placed into a
single work item and all this data ends up in a network buffer between the
Runner and the SDK so splitting the GBK output never happens.

You'll need to use runner v2 to get around this issue since it is the only
one that solves the buffered splitting problem. Runner v1 only supports
splitting the "unread" portion of the GBK while runner v2 supports
splitting everything from the current element being processed including
what has been buffered after it in addition to the "unread" portion of the
GBK.



On Fri, Aug 21, 2020 at 1:54 PM Jiadai Xia <da...@google.com> wrote:

> [image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
> When I check into the reshuffle before the processing, it seems that the
> GroupbyKey function output one element each time and wait until the
> processing part finish reading the element and then it start to output the
> next element.
> Is that the correct order of operation or is it altered by the dataflow
> fusion? I have set the num of workers to be 3 and disabled the autoscaling.
>
> On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:
>
>> Yes it does.
>>
>> There should be a reshuffle between the initial splitting and the
>> processing portion.
>>
>> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com> wrote:
>>
>>> I am using v1. Does v1 support the initial splitting and distribution?
>>> since I expect it to distribute the initial splitting to multiple workers.
>>>
>>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>>
>>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>>> still uses Dataflow runner v1?
>>>> Dataflow runner v2 is the only one that supports autoscaling and
>>>> dynamic splitting of splittable dofns in bounded pipelines.
>>>>
>>>> 1:
>>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>>
>>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>>> wrote:
>>>>
>>>>> Hi,
>>>>> As stated in the title, I tried to implement a SDF for reading the
>>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>>> I have tested it with Direct runner and the parallelism looks fine on
>>>>> small samples on Direct Runner.
>>>>> Below is my implementation of the SDF
>>>>> https://github.com/apache/beam/pull/12223
>>>>> --
>>>>>
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> *Jiadai Xia*
>>>>>
>>>>> SWE Intern
>>>>>
>>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>>
>>>>> danielxjd@google.com
>>>>>
>>>>> <https://www.linkedin.com/company/google/>
>>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>>> <https://www.facebook.com/lifeatgoogle/>
>>>>> <https://twitter.com/lifeatgoogle>
>>>>>
>>>>> <https://www.instagram.com/lifeatgoogle>
>>>>>
>>>>>
>>>>>
>>>
>>> --
>>>
>>>
>>>
>>>
>>>
>>> *Jiadai Xia*
>>>
>>> SWE Intern
>>>
>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>
>>> danielxjd@google.com
>>>
>>> <https://www.linkedin.com/company/google/>
>>> <https://www.youtube.com/user/lifeatgoogle>
>>> <https://www.facebook.com/lifeatgoogle/>
>>> <https://twitter.com/lifeatgoogle>
>>>
>>> <https://www.instagram.com/lifeatgoogle>
>>>
>>>
>>>
>
> --
>
>
>
>
>
> *Jiadai Xia*
>
> SWE Intern
>
> 1 (646) 413 8071 <(646)%20413-8071>
>
> danielxjd@google.com
>
> <https://www.linkedin.com/company/google/>
> <https://www.youtube.com/user/lifeatgoogle>
> <https://www.facebook.com/lifeatgoogle/>
> <https://twitter.com/lifeatgoogle>
>
> <https://www.instagram.com/lifeatgoogle>
>
>
>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Jiadai Xia <da...@google.com>.
[image: Screen Shot 2020-08-21 at 1.48.14 PM.png]
When I check into the reshuffle before the processing, it seems that the
GroupbyKey function output one element each time and wait until the
processing part finish reading the element and then it start to output the
next element.
Is that the correct order of operation or is it altered by the dataflow
fusion? I have set the num of workers to be 3 and disabled the autoscaling.

On Fri, Aug 21, 2020 at 11:45 AM Luke Cwik <lc...@google.com> wrote:

> Yes it does.
>
> There should be a reshuffle between the initial splitting and the
> processing portion.
>
> On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com> wrote:
>
>> I am using v1. Does v1 support the initial splitting and distribution?
>> since I expect it to distribute the initial splitting to multiple workers.
>>
>> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>>
>>> Are you using Dataflow runner v2[1] since the default for Beam Java
>>> still uses Dataflow runner v1?
>>> Dataflow runner v2 is the only one that supports autoscaling and dynamic
>>> splitting of splittable dofns in bounded pipelines.
>>>
>>> 1:
>>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>>
>>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com>
>>> wrote:
>>>
>>>> Hi,
>>>> As stated in the title, I tried to implement a SDF for reading the
>>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>>> and the work is not distributed. Any suggestion on what can be the problem?
>>>> I have tested it with Direct runner and the parallelism looks fine on
>>>> small samples on Direct Runner.
>>>> Below is my implementation of the SDF
>>>> https://github.com/apache/beam/pull/12223
>>>> --
>>>>
>>>>
>>>>
>>>>
>>>>
>>>> *Jiadai Xia*
>>>>
>>>> SWE Intern
>>>>
>>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>>
>>>> danielxjd@google.com
>>>>
>>>> <https://www.linkedin.com/company/google/>
>>>> <https://www.youtube.com/user/lifeatgoogle>
>>>> <https://www.facebook.com/lifeatgoogle/>
>>>> <https://twitter.com/lifeatgoogle>
>>>>
>>>> <https://www.instagram.com/lifeatgoogle>
>>>>
>>>>
>>>>
>>
>> --
>>
>>
>>
>>
>>
>> *Jiadai Xia*
>>
>> SWE Intern
>>
>> 1 (646) 413 8071 <(646)%20413-8071>
>>
>> danielxjd@google.com
>>
>> <https://www.linkedin.com/company/google/>
>> <https://www.youtube.com/user/lifeatgoogle>
>> <https://www.facebook.com/lifeatgoogle/>
>> <https://twitter.com/lifeatgoogle>
>>
>> <https://www.instagram.com/lifeatgoogle>
>>
>>
>>

-- 





*Jiadai Xia*

SWE Intern

1 (646) 413 8071 <(646)%20413-8071>

danielxjd@google.com

<https://www.linkedin.com/company/google/>
<https://www.youtube.com/user/lifeatgoogle>
<https://www.facebook.com/lifeatgoogle/> <https://twitter.com/lifeatgoogle>

<https://www.instagram.com/lifeatgoogle>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Luke Cwik <lc...@google.com>.
Yes it does.

There should be a reshuffle between the initial splitting and the
processing portion.

On Fri, Aug 21, 2020 at 11:04 AM Jiadai Xia <da...@google.com> wrote:

> I am using v1. Does v1 support the initial splitting and distribution?
> since I expect it to distribute the initial splitting to multiple workers.
>
> On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:
>
>> Are you using Dataflow runner v2[1] since the default for Beam Java still
>> uses Dataflow runner v1?
>> Dataflow runner v2 is the only one that supports autoscaling and dynamic
>> splitting of splittable dofns in bounded pipelines.
>>
>> 1:
>> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>>
>> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com> wrote:
>>
>>> Hi,
>>> As stated in the title, I tried to implement a SDF for reading the
>>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>>> split outputs a bunch of ranges but the number of workers are not scaled up
>>> and the work is not distributed. Any suggestion on what can be the problem?
>>> I have tested it with Direct runner and the parallelism looks fine on
>>> small samples on Direct Runner.
>>> Below is my implementation of the SDF
>>> https://github.com/apache/beam/pull/12223
>>> --
>>>
>>>
>>>
>>>
>>>
>>> *Jiadai Xia*
>>>
>>> SWE Intern
>>>
>>> 1 (646) 413 8071 <(646)%20413-8071>
>>>
>>> danielxjd@google.com
>>>
>>> <https://www.linkedin.com/company/google/>
>>> <https://www.youtube.com/user/lifeatgoogle>
>>> <https://www.facebook.com/lifeatgoogle/>
>>> <https://twitter.com/lifeatgoogle>
>>>
>>> <https://www.instagram.com/lifeatgoogle>
>>>
>>>
>>>
>
> --
>
>
>
>
>
> *Jiadai Xia*
>
> SWE Intern
>
> 1 (646) 413 8071 <(646)%20413-8071>
>
> danielxjd@google.com
>
> <https://www.linkedin.com/company/google/>
> <https://www.youtube.com/user/lifeatgoogle>
> <https://www.facebook.com/lifeatgoogle/>
> <https://twitter.com/lifeatgoogle>
>
> <https://www.instagram.com/lifeatgoogle>
>
>
>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Jiadai Xia <da...@google.com>.
I am using v1. Does v1 support the initial splitting and distribution?
since I expect it to distribute the initial splitting to multiple workers.

On Fri, Aug 21, 2020 at 11:00 AM Luke Cwik <lc...@google.com> wrote:

> Are you using Dataflow runner v2[1] since the default for Beam Java still
> uses Dataflow runner v1?
> Dataflow runner v2 is the only one that supports autoscaling and dynamic
> splitting of splittable dofns in bounded pipelines.
>
> 1:
> https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2
>
> On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com> wrote:
>
>> Hi,
>> As stated in the title, I tried to implement a SDF for reading the
>> Parquet file and I am trying to run it with Dataflow runner. As the initial
>> split outputs a bunch of ranges but the number of workers are not scaled up
>> and the work is not distributed. Any suggestion on what can be the problem?
>> I have tested it with Direct runner and the parallelism looks fine on
>> small samples on Direct Runner.
>> Below is my implementation of the SDF
>> https://github.com/apache/beam/pull/12223
>> --
>>
>>
>>
>>
>>
>> *Jiadai Xia*
>>
>> SWE Intern
>>
>> 1 (646) 413 8071 <(646)%20413-8071>
>>
>> danielxjd@google.com
>>
>> <https://www.linkedin.com/company/google/>
>> <https://www.youtube.com/user/lifeatgoogle>
>> <https://www.facebook.com/lifeatgoogle/>
>> <https://twitter.com/lifeatgoogle>
>>
>> <https://www.instagram.com/lifeatgoogle>
>>
>>
>>

-- 





*Jiadai Xia*

SWE Intern

1 (646) 413 8071

danielxjd@google.com

<https://www.linkedin.com/company/google/>
<https://www.youtube.com/user/lifeatgoogle>
<https://www.facebook.com/lifeatgoogle/> <https://twitter.com/lifeatgoogle>

<https://www.instagram.com/lifeatgoogle>

Re: Splittable-Dofn not distributing the work to multiple workers

Posted by Luke Cwik <lc...@google.com>.
Are you using Dataflow runner v2[1] since the default for Beam Java still
uses Dataflow runner v1?
Dataflow runner v2 is the only one that supports autoscaling and dynamic
splitting of splittable dofns in bounded pipelines.

1:
https://cloud.google.com/dataflow/docs/guides/deploying-a-pipeline#dataflow-runner-v2

On Fri, Aug 21, 2020 at 10:54 AM Jiadai Xia <da...@google.com> wrote:

> Hi,
> As stated in the title, I tried to implement a SDF for reading the Parquet
> file and I am trying to run it with Dataflow runner. As the initial split
> outputs a bunch of ranges but the number of workers are not scaled up and
> the work is not distributed. Any suggestion on what can be the problem?
> I have tested it with Direct runner and the parallelism looks fine on
> small samples on Direct Runner.
> Below is my implementation of the SDF
> https://github.com/apache/beam/pull/12223
> --
>
>
>
>
>
> *Jiadai Xia*
>
> SWE Intern
>
> 1 (646) 413 8071 <(646)%20413-8071>
>
> danielxjd@google.com
>
> <https://www.linkedin.com/company/google/>
> <https://www.youtube.com/user/lifeatgoogle>
> <https://www.facebook.com/lifeatgoogle/>
> <https://twitter.com/lifeatgoogle>
>
> <https://www.instagram.com/lifeatgoogle>
>
>
>