Re: [EXT] Re: [EXT] Re: [EXT] Re: [EXT] Re: [EXT] Re: Beam Dataframe - sort and grouping

2021-05-14 Thread Wenbing Bai
Thank you for the clarification! Is there a way to control the number of
shards, i.e. the bundle? I know in pure Beam IO connectors, we have
num_shards supported, for example, WriteToParquet
<https://github.com/apache/beam/blob/979269e4519bbe46105f68e0b8aa566bf854d95d/sdks/python/apache_beam/io/parquetio.py#L344>
.

Wenbing

On Thu, May 13, 2021 at 4:35 PM Robert Bradshaw  wrote:

> Sharding is determined by the distribution of work. Each worker writes to
> its own shard, and in the case of dynamic partitioning, etc. workers may
> end up processing more than one "bundle" of items and hence produce more
> than one shard. See also
> https://beam.apache.org/documentation/runtime/model/
>
> On Thu, May 13, 2021 at 3:58 PM Wenbing Bai 
> wrote:
>
>> Hi team,
>>
>> I have another question when using Beam Dataframe IO connector. I tried
>> to_parquet, and my data are written to several different files. I am
>> wondering how I can control the number of files (shards) or how the
>> sharding is done for to_parquet and other Beam Dataframe IO APIs?
>>
>> Thank you!
>> Wenbing
>>
>> On Tue, May 11, 2021 at 12:20 PM Kenneth Knowles  wrote:
>>
>>> +dev 
>>>
>>> In the Beam Java ecosystem, this functionality is provided by the Sorter
>>> library (
>>> https://beam.apache.org/documentation/sdks/java-extensions/#sorter).
>>> I'm curious what people think about various options:
>>>
>>>  - Python version of the transform(s)
>>>  - Expose sorter as xlang transform(s)
>>>  - Convenience transforms (that use pandas in DoFns?) to just do it for
>>> small data per key to achieve compatibility
>>>  - Beam model extension so that runners can do it as part of GBK
>>>
>>> Kenn
>>>
>>> On Mon, May 10, 2021 at 5:26 PM Wenbing Bai 
>>> wrote:
>>>
>>>> Hi Robert and Brian,
>>>>
>>>> I don't know why I didn't catch your replies. But thank you so much for
>>>> looking at this.
>>>>
>>>> My parquet files will be consumed by downstreaming processes which
>>>> require data points with the same "key1" that are sorted by "key2". The
>>>> downstreaming process, for example, will make a rolling window with size N
>>>> that reads N records together at one time. But note, the rolling window
>>>> will not cross different "key1".
>>>>
>>>> So that is saying, 1) I don't need to sort the whole dataset. 2) all
>>>> data with the same "key1" should be located together.
>>>>
>>>> I am not sure if I explain the use case clearly. Let me know what you
>>>> think.
>>>>
>>>> Wenbing
>>>>
>>>>
>>>> On Tue, Apr 20, 2021 at 5:01 PM Robert Bradshaw 
>>>> wrote:
>>>>
>>>>> It would also be helpful to understand what your overall objective is
>>>>> with this output. Is there a reason you need it sorted/partitioned in a
>>>>> certain way?
>>>>>
>>>>> On Tue, Apr 20, 2021 at 4:51 PM Brian Hulette 
>>>>> wrote:
>>>>>
>>>>>> Hi Wenbing,
>>>>>> Sorry for taking so long to get back to you on this.
>>>>>> I discussed this with Robert offline and we came up with a potential
>>>>>> workaround - you could try writing out the Parquet file from within the
>>>>>> groupby.apply method. You can use beam's FileSystems abstraction to open 
>>>>>> a
>>>>>> Python file object referencing a cloud storage file, and pass that file
>>>>>> object directly to the pandas to_parquet. It would look something like 
>>>>>> this:
>>>>>>
>>>>>>   df.groupby('key1').apply(lambda df:
>>>>>> df.sort_values(by='key2').to_parquet(FileSystems.open("gs://bucket/file.pq"))
>>>>>>
>>>>>> If writing out sorted, partitioned parquet files is a common use-case
>>>>>> we should think about making this easier though. At the very least
>>>>>> partition_cols should work, I filed BEAM-12201 [1] for this. That alone
>>>>>> won't be enough as our implementation will likely reshuffle the dataset 
>>>>>> to
>>>>>> enforce the partitioning, removing any sorting that you've applied, so 
>>>>>> we'd
>>>>>> also need to think about how to optimize the pi

Re: [EXT] Re: [EXT] Re: [EXT] Re: [EXT] Re: Beam Dataframe - sort and grouping

2021-05-13 Thread Wenbing Bai
Hi team,

I have another question when using Beam Dataframe IO connector. I tried
to_parquet, and my data are written to several different files. I am
wondering how I can control the number of files (shards) or how the
sharding is done for to_parquet and other Beam Dataframe IO APIs?

Thank you!
Wenbing

On Tue, May 11, 2021 at 12:20 PM Kenneth Knowles  wrote:

> +dev 
>
> In the Beam Java ecosystem, this functionality is provided by the Sorter
> library (
> https://beam.apache.org/documentation/sdks/java-extensions/#sorter). I'm
> curious what people think about various options:
>
>  - Python version of the transform(s)
>  - Expose sorter as xlang transform(s)
>  - Convenience transforms (that use pandas in DoFns?) to just do it for
> small data per key to achieve compatibility
>  - Beam model extension so that runners can do it as part of GBK
>
> Kenn
>
> On Mon, May 10, 2021 at 5:26 PM Wenbing Bai 
> wrote:
>
>> Hi Robert and Brian,
>>
>> I don't know why I didn't catch your replies. But thank you so much for
>> looking at this.
>>
>> My parquet files will be consumed by downstreaming processes which
>> require data points with the same "key1" that are sorted by "key2". The
>> downstreaming process, for example, will make a rolling window with size N
>> that reads N records together at one time. But note, the rolling window
>> will not cross different "key1".
>>
>> So that is saying, 1) I don't need to sort the whole dataset. 2) all data
>> with the same "key1" should be located together.
>>
>> I am not sure if I explain the use case clearly. Let me know what you
>> think.
>>
>> Wenbing
>>
>>
>> On Tue, Apr 20, 2021 at 5:01 PM Robert Bradshaw 
>> wrote:
>>
>>> It would also be helpful to understand what your overall objective is
>>> with this output. Is there a reason you need it sorted/partitioned in a
>>> certain way?
>>>
>>> On Tue, Apr 20, 2021 at 4:51 PM Brian Hulette 
>>> wrote:
>>>
>>>> Hi Wenbing,
>>>> Sorry for taking so long to get back to you on this.
>>>> I discussed this with Robert offline and we came up with a potential
>>>> workaround - you could try writing out the Parquet file from within the
>>>> groupby.apply method. You can use beam's FileSystems abstraction to open a
>>>> Python file object referencing a cloud storage file, and pass that file
>>>> object directly to the pandas to_parquet. It would look something like 
>>>> this:
>>>>
>>>>   df.groupby('key1').apply(lambda df:
>>>> df.sort_values(by='key2').to_parquet(FileSystems.open("gs://bucket/file.pq"))
>>>>
>>>> If writing out sorted, partitioned parquet files is a common use-case
>>>> we should think about making this easier though. At the very least
>>>> partition_cols should work, I filed BEAM-12201 [1] for this. That alone
>>>> won't be enough as our implementation will likely reshuffle the dataset to
>>>> enforce the partitioning, removing any sorting that you've applied, so we'd
>>>> also need to think about how to optimize the pipeline to avoid that 
>>>> shuffle.
>>>>
>>>> Brian
>>>>
>>>> [1] https://issues.apache.org/jira/browse/BEAM-12201
>>>>
>>>> On Wed, Apr 7, 2021 at 9:02 PM Wenbing Bai 
>>>> wrote:
>>>>
>>>>> Thank you, Brian. I tried `partition_cols`, but it is not working. I
>>>>> tried pure pandas, it does work, so I am not sure if anything wrong with
>>>>> Beam.
>>>>>
>>>>> Wenbing
>>>>>
>>>>> On Wed, Apr 7, 2021 at 2:56 PM Brian Hulette 
>>>>> wrote:
>>>>>
>>>>>> Hm, to_parquet does have a `partition_cols` argument [1] which we
>>>>>> pass through [2]. It would be interesting to see what
>>>>>> `partition_cols='key1'` does - I suspect it won't work perfectly though.
>>>>>>
>>>>>> Do you have any thoughts here Robert?
>>>>>>
>>>>>> [1]
>>>>>> https://pandas.pydata.org/pandas-docs/stable/reference/api/pandas.DataFrame.to_parquet.html
>>>>>> [2]
>>>>>> https://github.com/apache/beam/blob/a8cd05932bed9b2480316fb8518409636cb2733b/sdks/python/apache_beam/dataframe/io.py#L525
>>>>>>
>>>>>> On Wed, Apr 7, 2021 at 2:22 PM Wenbing Bai 
>>&

Re: Can any contributor trigger PreCommit tests?

2020-02-26 Thread Wenbing Bai
Hi Liu,

Here is my PR https://github.com/apache/beam/pull/10901. Pablo is helping
on this PR.

Thank you for the quick response.

Wenbing

On Tue, Feb 25, 2020 at 1:16 PM Liu Wang  wrote:

> How can I become a committer?
>
> On 2020/02/25 19:14:28, Hannah Jiang  wrote:
> > Committers can trigger the tests. Can you share your PRs here so people
> who
> > have permission can trigger tests for your PRs?
> > Please refer to this thread
> > <
> https://lists.apache.org/thread.html/27c1482cb02bdf03e8ea7ad48fe6d2c170527507ed13bed8dea87b7e%40%3Cdev.beam.apache.org%3E
> >
> > for more details of the issue.
> >
> >
> >
> >
> > On Tue, Feb 25, 2020 at 10:53 AM Wenbing Bai 
> > wrote:
> >
> > > I have the same issue. Follow this thread.
> > >
> > > Wenbing
> > >
> > > On Tue, Feb 25, 2020 at 9:43 AM Liu Wang  wrote:
> > >
> > >> Hi,
> > >>
> > >> I'm a contributor but can't trigger tests for my PRs. Do I need to get
> > >> any permission for this?
> > >> Is it like any contributor can apply for and get the permission?
> > >>
> > >> Thanks,
> > >> Liu
> > >>
> > >
> > >
> > > --
> > >
> > >
> > >
> > >
> > >
> > > Wenbing Bai
> > >
> > > Senior Software Engineer, MLP
> > >
> > > Cruise
> > >
> > > Pronouns: She/Her
> > >
> > >
> > >
> > > *Confidentiality Note:* We care about protecting our proprietary
> > > information, confidential material, and trade secrets. This message may
> > > contain some or all of those things. Cruise will suffer material harm
> if
> > > anyone other than the intended recipient disseminates or takes any
> action
> > > based on this message. If you have received this message (including any
> > > attachments) in error, please delete it immediately and notify the
> sender
> > > promptly.
> >
>


-- 





Wenbing Bai

Senior Software Engineer, MLP

Cruise

Pronouns: She/Her

-- 


*Confidentiality Note:* We care about protecting our proprietary 
information, confidential material, and trade secrets. This message may 
contain some or all of those things. Cruise will suffer material harm if 
anyone other than the intended recipient disseminates or takes any action 
based on this message. If you have received this message (including any 
attachments) in error, please delete it immediately and notify the sender 
promptly.


Re: Can any contributor trigger PreCommit tests?

2020-02-25 Thread Wenbing Bai
I have the same issue. Follow this thread.

Wenbing

On Tue, Feb 25, 2020 at 9:43 AM Liu Wang  wrote:

> Hi,
>
> I'm a contributor but can't trigger tests for my PRs. Do I need to get any
> permission for this?
> Is it like any contributor can apply for and get the permission?
>
> Thanks,
> Liu
>


-- 





Wenbing Bai

Senior Software Engineer, MLP

Cruise

Pronouns: She/Her

-- 


*Confidentiality Note:* We care about protecting our proprietary 
information, confidential material, and trade secrets. This message may 
contain some or all of those things. Cruise will suffer material harm if 
anyone other than the intended recipient disseminates or takes any action 
based on this message. If you have received this message (including any 
attachments) in error, please delete it immediately and notify the sender 
promptly.


Review Request

2020-02-24 Thread Wenbing Bai
Hi there,

Can I get a review on this PR https://github.com/apache/beam/pull/10901?

-- 





Wenbing Bai

Senior Software Engineer, MLP

Cruise

Pronouns: She/Her

-- 


*Confidentiality Note:* We care about protecting our proprietary 
information, confidential material, and trade secrets. This message may 
contain some or all of those things. Cruise will suffer material harm if 
anyone other than the intended recipient disseminates or takes any action 
based on this message. If you have received this message (including any 
attachments) in error, please delete it immediately and notify the sender 
promptly.


Contributor permission for Beam Jira tickets

2020-02-13 Thread Wenbing Bai
Hi there,

I am Wenbing from Cruise. I would like to make some contributions to
the Python SDK for Beam. Can someone add me as a contributor in the
Beam Jira? My username is wenbing-bai.

Thank you!
Wenbing

-- 





Wenbing Bai

Senior Software Engineer, MLP

Cruise

Pronouns: She/Her

-- 


*Confidentiality Note:* We care about protecting our proprietary 
information, confidential material, and trade secrets. This message may 
contain some or all of those things. Cruise will suffer material harm if 
anyone other than the intended recipient disseminates or takes any action 
based on this message. If you have received this message (including any 
attachments) in error, please delete it immediately and notify the sender 
promptly.