Re: Can we solve WindowFn.getOutputTime another way?

2021-02-11 Thread Robert Bradshaw
On Wed, Feb 10, 2021 at 8:03 PM Kenneth Knowles  wrote:

>
>
> On Wed, Feb 10, 2021 at 2:24 PM Alex Amato  wrote:
>
>>
>>
>> On Wed, Feb 10, 2021 at 12:14 PM Kenneth Knowles  wrote:
>>
>>> On a PR (https://github.com/apache/beam/pull/13927) we got into a
>>> discussion of a very old and strange feature of Beam that I think we should
>>> revisit.
>>>
>>> The WindowFn has the ability to shift timestamps forward in order to
>>> unblock downstream watermarks. Why? Specifically in this situation:
>>>
>>>  - aggregation/GBK with overlapping windows like SlidingWindows
>>>  - timestamp combiner of the aggregated outputs is EARLIEST of the inputs
>>>  - there is another downstream aggregation/GBK
>>>
>>> The output watermark of the upstream aggregation is held to the minimum
>>> of the inputs. When an output is emitted, we desire the output to flow
>>> through the rest of the pipeline without delay. However, the downstream
>>> aggregation can (and often will) be delayed by the window size because of 
>>> *watermark
>>> holds in other later windows that are not released until those windows
>>> output.*
>>>
>> Could you describe this a bit more? Why would later windows hold up the
>> watermark for upstream steps. (Is it due to some subtlety? Such as tracking
>> the watermark for each stage, rather than for each step?)
>>
>
> It does not have to do with stages/fusion (a runner-specific concept) but
> is a necessity of watermarks being per-PCollection.
>
> Suppose:
>
>  - Default triggering
>  - Timestamp combiner EARLIEST
>  - 60s windows sliding every 10s
>  - An element with timestamp 42
>  - Aggregation (A) with downstream aggregation (B)
>
> Here is what happens:
>
>  - The element falls into [-10, 50) and [0, 60) and [10, 70) and [20, 80)
> and [30, 90) and [40, 100)
>  - For each of those windows the output watermark hold is set to 42 (the
> element's timestamp)
>  - At time 50 the aggregation (A) over the first window is emitted; the
> other windows remain buffered and held
>  - The element arrives at aggregation (B) and is buffered because the
> input watermark (which is the held output watermark from A) is still 42,
> even though no other data will arrive for that window (WLOG if elements
> from other keys are shuffled in)
>  - The input watermark for aggregation (B) does not advance past 42 until
> the [40, 100) window is fired and releases its watermark hold
>
> It is, indeed, subtle. To me, anyhow. I was wrong - it is not delayed by
> the window size, but by the difference in end-of-window timestamps to all
> assigned windows (window size minus slide?)
>
> So to avoid this, what actually happens in Java today is that the
> watermark hold, and output timestamp, is set not to 42 but altered to 50 to
> not overlap the prior window. Timestamp of 50 is very nonintuitive since
> you asked for the EARLIEST of input timestamps. EARLIEST combiner plays an
> important role in CoGBK based joins in SQL, where the iterables are
> re-exploded with timestamps that may be the minimum of input elements. This
> shifting may actually break SQL...
>
> This predated our switch away from "delta from watermark" late data
> dropping to "window expiry" data dropping. So maybe there is some new way
> to set a hold that does not make data late or droppable but still use the
> EARLIEST timestamp. That is my question, for which I have not figured out
> the answer.
>

This is, indeed, a very tough question...

I'd say this is generally a problem with EARLIEST and non-aligned windows.
E.g. for sessions, a long key can hold up the watermark for all others.
Here we "know" what the hold up is, and can adjust for it. But I don't
think doing this adjustment is the right thing. It would certainly seem to
mess up the timestamp of the outputs from a join. And it's possible that
the values get re-windowed in which case this element should get joined
with itself from a later window (which I'll admit is a bit odd, but maybe a
reflection that multiple-windowing, like multi-firing triggering, is
non-local).

Logicaly, the reason we want [-10 50) window for B to fire shortly after
the input watermark for A passes 50 because no non-late data coming out of
A could influence it. In some sense, the "watermark" for the [-10, 50)
windows has indeed passed, but not that for later windows. I don't think
the beam model requires that we have a single watermark, just that we fire
triggers/timers once we have seen all the on-time data that we think we
could, and a runner could be smart about this.

We may want to keep the ability to shift timestamps for WindowFns, but I
think we shouldn't be doing so for the default sliding windows. Correctness
(of output timestamps) over latency unless one asks otherwise.


> Kenn
>
>
>>
>>> To avoid this problem, element x in window w will have its timestamp
>>> shifted to not overlap with any earlier windows. It is a weird behavior. It
>>> fixes the watermark hold problem but introduces a strange output with a
>>> mysterious

Re: Do we need synchronized processing time? / What to do about "continuation triggers"?

2021-02-11 Thread Robert Bradshaw
Of course the right answer is to just implement sink triggers and sidestep
the question altogether :).

In the meantime, I think leaving AfterSynchronizedProcessingTime in the
model makes the most sense, and runners can choose an implementation
between firing eagerly and waiting some amount of time until they think all
(most?) downstream results are in before firing, depending on how smart the
runner wants to be. As you point out, they're all correct, and we'll have
multiple firings due to the upstream trigger anyway, and this is safer than
it used to be (though still possibly requires work).


On Wed, Feb 10, 2021 at 1:37 PM Kenneth Knowles  wrote:

> Hi all,
>
> TL;DR:
> 1. should we replace "after synchronized processing time" with "after
> count 1"?
> 2. should we remove "continuation trigger" and leave this to runners?
>
> 
>
> "AfterSynchronizedProcessingTime" triggers were invented to solve a
> specific problem. They are inconsistent across SDKs today.
>
>  - You have an aggregation/GBK with aligned processing time trigger like
> ("output every minute on the minute")
>  - You have a downstream aggregation/GBK between that and the sink
>  - You expect to have about one output every minute per key+window pair
>
> Any output of the upstream aggregation may contribute to any key+window of
> the downstream aggregation. The AfterSynchronizedProcessingTime trigger
> waits for all the processing time based triggers to fire and commit their
> outputs. The downstream aggregation will output as fast as possible in
> panes consistent with the upstream aggregation.
>
>  - The Java SDK behavior is as above, to output "as fast as reasonable".
>  - The Python SDK never uses "AfterSynchronizedProcessingTime" triggers
> but simply propagates the same trigger to the next GBK, creating additional
> delay.
>  - I don't know what the Go SDK may do, if it supports this at all.
>
> Any behavior could be defined as "correct". A simple option could be to
> have the downstream aggregation "fire always" aka "after element count 1".
> How would this change things? We would potentially see many more outputs.
>
> Why did we do this in the first place? There are (at least) these reasons:
>
>  - Previously, triggers could "finish" an aggregation thus dropping all
> further data. In this case, waiting for all outputs is critical or else you
> lose data. Now triggers cannot finish aggregations.
>  - Whenever there may be more than one pane, a user has to write logic to
> compensate and deal with it. Changing from guaranteed single pane to
> multi-pane would break things. So if the user configures a single firing,
> all downstream aggregations must respect it. Now that triggers cannot
> finish, I think processing time can only be used in multi-pane contexts
> anyhow.
>  - The above example illustrates how the behavior in Java maintains
> something that the user will expect. Or so we think. Maybe users don't care.
>
> How did we get into this inconsistent state? When the user specifies
> triggering it applies to the very nearest aggregation/GBK. The SDK decides
> what triggering to insert downstream. One possibility is to remove this and
> have it unspecified, left to runner behavior.
>
> I think maybe these pieces of complexity are both not helpful and also not
> (necessarily) breaking changes to alter, especially considering we have
> inconsistency in the model.
>
> WDYT? And I wonder what this means for xlang and portability... how does
> continuation triggering even work? (if at all)
>
> Kenn
>


Pull Request Review / Feature Feedback for KafkaIO

2021-02-11 Thread Rion Williams
Hi all,

Recently, I encountered a bit of functionality for a pipeline that I was
working that seemed to be slightly lacking (specifically the recognition of
explicitly defined partitioning in the KafkaIO.WriteRecords transform) so I
put together a JIRA related to it [1] as well as a more detailed pull
request [1] with an initial potential fix/change.

I'll provide a bit more context from the pull request description below in
case in-thread feedback would be easier for some, but any
recommendations/reviewers/advice would be greatly appreciated!

Cheers,

Rion

[1]: https://issues.apache.org/jira/browse/BEAM-11806
[2]: https://github.com/apache/beam/pull/13975

--

At present, the WriteRecords transform for KafkaIO does not recognize the
partition property defined on ProducerRecord instances consumed by the
transform:

producer.send(
// The null property in the following constructor represents partition
new ProducerRecord<>(
topicName, null, timestampMillis, record.key(),
record.value(), record.headers()),
new SendCallback());

Because of this limitation, in a scenario where a user may desire an
explicitly defined partitioning strategy as opposed to round-robin, they
would have to create their own custom DoFn that defines a KafkaProducer
(preferably within a @StartBundle) similar to the following approach (in
Kotlin):

private class ExampleProducerDoFn(...): DoFn<...>() {
private lateinit var producer: KafkaProducer<...>

@StartBundle
fun startBundle(context: StartBundleContext) {
val options =
context.pipelineOptions.`as`(YourPipelineOptions::class.java)
producer = getKafkaProducer(options)
}

@ProcessElement
fun processElement(context: ProcessContext){
// Omitted for brevity

// Produce the record to a specific topic at a specific partition
producer.send(ProducerRecord(
"your_topic_here",
your_partition_here,
context.element().kv.key,
context.element().kv.value
))
}

The *initial* pull request that I threw in here simply replaces the
existing null with the record.partition() (i.e. the record that was
explicitly defined initially, but it may require some other changes which
I'd need someone more familiar with the KafkaIO source to chime in on.


Re: [Proposal] Requesting PMC approval to start planning for Beam Summits 2021

2021-02-11 Thread Brittany Hermann
Nothing substantial. Only change is going from a half day, five day event
to a 3 day full day event.

On Thu, Feb 11, 2021 at 7:24 PM Robert Bradshaw  wrote:

> Are there any substantive changes from what we did last year?
>
> On Thu, Feb 11, 2021 at 1:40 PM Brittany Hermann 
> wrote:
>
>> Dear Project Management Committee,
>>
>> The Beam Summit is a community event funded by a group of sponsors and
>> organized by a steering committee formed by members of the Beam community
>> and who have participated in past editions. I'd like to get the following
>> approvals:
>>
>> -To organize and host the summit under the name of Beam Summit 
>> , i.e. Digital Beam Summit 2021.
>>
>> -Approval to host this year's edition on the following dates: Wednesday,
>> August 4th - Friday, August 6th, 2021. Digital Beam Summit, an online,
>> multi-day event to be hosted in a selected video conference platform, and
>> to be organized in the date range of the beginning of August. Expected
>> attendees: 400 attendees live
>>
>> The event will provide educational content selected by the steering
>> committee, and will be a not-for-profit venture. This will be a free event.
>> The event will be advertised on various channels, including the Apache
>> Beam's and Summit sponsor's social media accounts.
>>
>> The organizing committee will acknowledge the Apache Software
>> Foundation's ownership of the Apache Beam trademark and will add
>> attribution required by the foundation's policy on all marketing channels.
>> The Apache Beam branding will be used in accordance with the foundation's
>> trademark and events policies specifically outlined in Third Party Event
>> Branding Policy. The organizing committee does not request the ASF to
>> become a community partner of the event.
>>
>> Attached is a full proposal with the event details for your reference
>> [1]. Please feel free to request further information if needed.
>>
>> Kind regards,
>>
>> -Brittany Hermann on behalf of the Beam Summit Steering Committee
>>
>> [1]
>>
>> https://docs.google.com/document/d/1nqE2DMfw2qsI57qIZzjF76L0W_NacxQFXIoZtDOApVY/edit?usp=sharing
>>
>>

-- 

Brittany Hermann

Open Source Program Manager (Provided by Adecco Staffing)

1190 Bordeaux Drive , Building 4, Sunnyvale, CA 94089



Re: [Proposal] Requesting PMC approval to start planning for Beam Summits 2021

2021-02-11 Thread Robert Bradshaw
Are there any substantive changes from what we did last year?

On Thu, Feb 11, 2021 at 1:40 PM Brittany Hermann 
wrote:

> Dear Project Management Committee,
>
> The Beam Summit is a community event funded by a group of sponsors and
> organized by a steering committee formed by members of the Beam community
> and who have participated in past editions. I'd like to get the following
> approvals:
>
> -To organize and host the summit under the name of Beam Summit 
> , i.e. Digital Beam Summit 2021.
>
> -Approval to host this year's edition on the following dates: Wednesday,
> August 4th - Friday, August 6th, 2021. Digital Beam Summit, an online,
> multi-day event to be hosted in a selected video conference platform, and
> to be organized in the date range of the beginning of August. Expected
> attendees: 400 attendees live
>
> The event will provide educational content selected by the steering
> committee, and will be a not-for-profit venture. This will be a free event.
> The event will be advertised on various channels, including the Apache
> Beam's and Summit sponsor's social media accounts.
>
> The organizing committee will acknowledge the Apache Software Foundation's
> ownership of the Apache Beam trademark and will add attribution required by
> the foundation's policy on all marketing channels. The Apache Beam branding
> will be used in accordance with the foundation's trademark and events
> policies specifically outlined in Third Party Event Branding Policy. The
> organizing committee does not request the ASF to become a community partner
> of the event.
>
> Attached is a full proposal with the event details for your reference [1].
> Please feel free to request further information if needed.
>
> Kind regards,
>
> -Brittany Hermann on behalf of the Beam Summit Steering Committee
>
> [1]
>
> https://docs.google.com/document/d/1nqE2DMfw2qsI57qIZzjF76L0W_NacxQFXIoZtDOApVY/edit?usp=sharing
>
>


[Proposal] Requesting PMC approval to start planning for Beam Summits 2021

2021-02-11 Thread Brittany Hermann
Dear Project Management Committee,

The Beam Summit is a community event funded by a group of sponsors and
organized by a steering committee formed by members of the Beam community
and who have participated in past editions. I'd like to get the following
approvals:

-To organize and host the summit under the name of Beam Summit 
, i.e. Digital Beam Summit 2021.

-Approval to host this year's edition on the following dates: Wednesday,
August 4th - Friday, August 6th, 2021. Digital Beam Summit, an online,
multi-day event to be hosted in a selected video conference platform, and
to be organized in the date range of the beginning of August. Expected
attendees: 400 attendees live

The event will provide educational content selected by the steering
committee, and will be a not-for-profit venture. This will be a free event.
The event will be advertised on various channels, including the Apache
Beam's and Summit sponsor's social media accounts.

The organizing committee will acknowledge the Apache Software Foundation's
ownership of the Apache Beam trademark and will add attribution required by
the foundation's policy on all marketing channels. The Apache Beam branding
will be used in accordance with the foundation's trademark and events
policies specifically outlined in Third Party Event Branding Policy. The
organizing committee does not request the ASF to become a community partner
of the event.

Attached is a full proposal with the event details for your reference [1].
Please feel free to request further information if needed.

Kind regards,

-Brittany Hermann on behalf of the Beam Summit Steering Committee

[1]
https://docs.google.com/document/d/1nqE2DMfw2qsI57qIZzjF76L0W_NacxQFXIoZtDOApVY/edit?usp=sharing


Re: [PROPOSAL] Preparing for Beam 2.28.0 release

2021-02-11 Thread Nir Gazit
Sure! Thanks!
https://github.com/apache/beam/pull/13965

On Thu, Feb 11, 2021 at 7:53 PM Chamikara Jayalath 
wrote:

> Yeah, this seems like a critical fix. Please send a cherry-pick. Also
> please make sure that this did not break any post-commits after being
> merged.
>
> Thanks,
> Cham
>
> On Thu, Feb 11, 2021 at 9:40 AM Nir Gazit  wrote:
>
>> Hey,
>> I hope it's not too late but is it possible to Cherry-pick this PR as
>> well:
>> https://github.com/apache/beam/pull/13723
>>
>> It fixes a critical bug
>>  that currently
>> prevents the use of S3IO (and probably other IOs) on the Python SDK.
>>
>> Thanks
>> Nir
>>
>> On Wed, Feb 10, 2021 at 1:37 AM Chamikara Jayalath 
>> wrote:
>>
>>> Update:
>>> Cherry-picks are in. Validating the release branch here:
>>> https://github.com/apache/beam/pull/13941
>>> 
>>>
>>> Thanks,
>>> Cham
>>>
>>> On Wed, Feb 3, 2021 at 6:41 PM Chamikara Jayalath 
>>> wrote:
>>>
 Update: we are down to a single blocker and a fix is in review.

 Thanks,
 Cham

 On Fri, Jan 29, 2021 at 5:51 PM Chamikara Jayalath <
 chamik...@google.com> wrote:

> Update:
>
> We have two remaining blockers: https://s.apache.org/beam-2.28.0
> -burn-down
> I hope to build the first RC when the blockers are resolved.
>
> Thanks,
> Cham
>
>
> On Wed, Jan 27, 2021 at 12:51 PM Chamikara Jayalath <
> chamik...@google.com> wrote:
>
>> Release branch was cut:
>> https://github.com/apache/beam/tree/release-2.28.0
>>
>> I'll work with owners of current release blocking JIRAs to
>> cherry-pick fixes before building a RC.
>>
>> Thanks,
>> Cham
>>
>> On Mon, Jan 25, 2021 at 5:20 PM Chamikara Jayalath <
>> chamik...@google.com> wrote:
>>
>>> Hi All,
>>>
>>> I hope to cut the 2.28.0 release branch on 01/27/2021 as scheduled.
>>>
>>> Please see [1] for the current JIRA burn down for Beam 2.28.0
>>> release.
>>>
>>> If you own any issue in that list, please try to resolve them as
>>> soon as possible (or remove from the list if not critical for this 
>>> release).
>>>
>>> If you find any new issues that require a critical fix in the Beam
>>> 2.28.0 release, please create a JIRA and add to the burn down list by
>>> changing the Fix Version to 2.28.0.
>>>
>>> Thanks,
>>> Cham
>>>
>>> [1] https://s.apache.org/beam-2.28.0-burn-down
>>>
>>>
>>>
>>> On Thu, Jan 14, 2021 at 4:41 PM Ankur Goenka 
>>> wrote:
>>>
 Thanks Cham!

 On Thu, Jan 14, 2021 at 12:26 PM Yichi Zhang 
 wrote:

> Thank you, Cham!
>
> On Wed, Jan 13, 2021 at 12:58 PM Ahmet Altay 
> wrote:
>
>> Thank you Cham!
>>
>> On Wed, Jan 13, 2021 at 12:44 PM Rui Wang 
>> wrote:
>>
>>> Thanks Cham for working on this!
>>>
>>>
>>> -Rui
>>>
>>> On Wed, Jan 13, 2021 at 11:32 AM Kyle Weaver <
>>> kcwea...@google.com> wrote:
>>>
 Thanks for stepping up Cham!

 Remember to mark critical JIRA issues as release blockers
 everybody!

 On Wed, Jan 13, 2021 at 11:25 AM Chamikara Jayalath <
 chamik...@google.com> wrote:

> Hi All,
>
> Beam 2.28.0 release is scheduled to be cut on January 27th
> according to the release calendar [1]
>
> I'd like to volunteer myself to be the release manager for
> this release. I plan on cutting the release branch on the 
> scheduled date.
>
> Any comments or objections ?
>
> Thanks,
> Cham
>
> [1]
> https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com&ctz=America/Los_Angeles
>



Re: [PROPOSAL] Preparing for Beam 2.28.0 release

2021-02-11 Thread Chamikara Jayalath
Yeah, this seems like a critical fix. Please send a cherry-pick. Also
please make sure that this did not break any post-commits after being
merged.

Thanks,
Cham

On Thu, Feb 11, 2021 at 9:40 AM Nir Gazit  wrote:

> Hey,
> I hope it's not too late but is it possible to Cherry-pick this PR as well:
> https://github.com/apache/beam/pull/13723
>
> It fixes a critical bug 
> that currently prevents the use of S3IO (and probably other IOs) on the
> Python SDK.
>
> Thanks
> Nir
>
> On Wed, Feb 10, 2021 at 1:37 AM Chamikara Jayalath 
> wrote:
>
>> Update:
>> Cherry-picks are in. Validating the release branch here:
>> https://github.com/apache/beam/pull/13941
>> 
>>
>> Thanks,
>> Cham
>>
>> On Wed, Feb 3, 2021 at 6:41 PM Chamikara Jayalath 
>> wrote:
>>
>>> Update: we are down to a single blocker and a fix is in review.
>>>
>>> Thanks,
>>> Cham
>>>
>>> On Fri, Jan 29, 2021 at 5:51 PM Chamikara Jayalath 
>>> wrote:
>>>
 Update:

 We have two remaining blockers: https://s.apache.org/beam-2.28.0
 -burn-down
 I hope to build the first RC when the blockers are resolved.

 Thanks,
 Cham


 On Wed, Jan 27, 2021 at 12:51 PM Chamikara Jayalath <
 chamik...@google.com> wrote:

> Release branch was cut:
> https://github.com/apache/beam/tree/release-2.28.0
>
> I'll work with owners of current release blocking JIRAs to cherry-pick
> fixes before building a RC.
>
> Thanks,
> Cham
>
> On Mon, Jan 25, 2021 at 5:20 PM Chamikara Jayalath <
> chamik...@google.com> wrote:
>
>> Hi All,
>>
>> I hope to cut the 2.28.0 release branch on 01/27/2021 as scheduled.
>>
>> Please see [1] for the current JIRA burn down for Beam 2.28.0 release.
>>
>> If you own any issue in that list, please try to resolve them as soon
>> as possible (or remove from the list if not critical for this release).
>>
>> If you find any new issues that require a critical fix in the Beam
>> 2.28.0 release, please create a JIRA and add to the burn down list by
>> changing the Fix Version to 2.28.0.
>>
>> Thanks,
>> Cham
>>
>> [1] https://s.apache.org/beam-2.28.0-burn-down
>>
>>
>>
>> On Thu, Jan 14, 2021 at 4:41 PM Ankur Goenka 
>> wrote:
>>
>>> Thanks Cham!
>>>
>>> On Thu, Jan 14, 2021 at 12:26 PM Yichi Zhang 
>>> wrote:
>>>
 Thank you, Cham!

 On Wed, Jan 13, 2021 at 12:58 PM Ahmet Altay 
 wrote:

> Thank you Cham!
>
> On Wed, Jan 13, 2021 at 12:44 PM Rui Wang 
> wrote:
>
>> Thanks Cham for working on this!
>>
>>
>> -Rui
>>
>> On Wed, Jan 13, 2021 at 11:32 AM Kyle Weaver 
>> wrote:
>>
>>> Thanks for stepping up Cham!
>>>
>>> Remember to mark critical JIRA issues as release blockers
>>> everybody!
>>>
>>> On Wed, Jan 13, 2021 at 11:25 AM Chamikara Jayalath <
>>> chamik...@google.com> wrote:
>>>
 Hi All,

 Beam 2.28.0 release is scheduled to be cut on January 27th
 according to the release calendar [1]

 I'd like to volunteer myself to be the release manager for this
 release. I plan on cutting the release branch on the scheduled 
 date.

 Any comments or objections ?

 Thanks,
 Cham

 [1]
 https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com&ctz=America/Los_Angeles

>>>


Re: [PROPOSAL] Preparing for Beam 2.28.0 release

2021-02-11 Thread Nir Gazit
Hey,
I hope it's not too late but is it possible to Cherry-pick this PR as well:
https://github.com/apache/beam/pull/13723

It fixes a critical bug 
that currently prevents the use of S3IO (and probably other IOs) on the
Python SDK.

Thanks
Nir

On Wed, Feb 10, 2021 at 1:37 AM Chamikara Jayalath 
wrote:

> Update:
> Cherry-picks are in. Validating the release branch here:
> https://github.com/apache/beam/pull/13941
> 
>
> Thanks,
> Cham
>
> On Wed, Feb 3, 2021 at 6:41 PM Chamikara Jayalath 
> wrote:
>
>> Update: we are down to a single blocker and a fix is in review.
>>
>> Thanks,
>> Cham
>>
>> On Fri, Jan 29, 2021 at 5:51 PM Chamikara Jayalath 
>> wrote:
>>
>>> Update:
>>>
>>> We have two remaining blockers: https://s.apache.org/beam-2.28.0
>>> -burn-down
>>> I hope to build the first RC when the blockers are resolved.
>>>
>>> Thanks,
>>> Cham
>>>
>>>
>>> On Wed, Jan 27, 2021 at 12:51 PM Chamikara Jayalath <
>>> chamik...@google.com> wrote:
>>>
 Release branch was cut:
 https://github.com/apache/beam/tree/release-2.28.0

 I'll work with owners of current release blocking JIRAs to cherry-pick
 fixes before building a RC.

 Thanks,
 Cham

 On Mon, Jan 25, 2021 at 5:20 PM Chamikara Jayalath <
 chamik...@google.com> wrote:

> Hi All,
>
> I hope to cut the 2.28.0 release branch on 01/27/2021 as scheduled.
>
> Please see [1] for the current JIRA burn down for Beam 2.28.0 release.
>
> If you own any issue in that list, please try to resolve them as soon
> as possible (or remove from the list if not critical for this release).
>
> If you find any new issues that require a critical fix in the Beam
> 2.28.0 release, please create a JIRA and add to the burn down list by
> changing the Fix Version to 2.28.0.
>
> Thanks,
> Cham
>
> [1] https://s.apache.org/beam-2.28.0-burn-down
>
>
>
> On Thu, Jan 14, 2021 at 4:41 PM Ankur Goenka 
> wrote:
>
>> Thanks Cham!
>>
>> On Thu, Jan 14, 2021 at 12:26 PM Yichi Zhang 
>> wrote:
>>
>>> Thank you, Cham!
>>>
>>> On Wed, Jan 13, 2021 at 12:58 PM Ahmet Altay 
>>> wrote:
>>>
 Thank you Cham!

 On Wed, Jan 13, 2021 at 12:44 PM Rui Wang 
 wrote:

> Thanks Cham for working on this!
>
>
> -Rui
>
> On Wed, Jan 13, 2021 at 11:32 AM Kyle Weaver 
> wrote:
>
>> Thanks for stepping up Cham!
>>
>> Remember to mark critical JIRA issues as release blockers
>> everybody!
>>
>> On Wed, Jan 13, 2021 at 11:25 AM Chamikara Jayalath <
>> chamik...@google.com> wrote:
>>
>>> Hi All,
>>>
>>> Beam 2.28.0 release is scheduled to be cut on January 27th
>>> according to the release calendar [1]
>>>
>>> I'd like to volunteer myself to be the release manager for this
>>> release. I plan on cutting the release branch on the scheduled date.
>>>
>>> Any comments or objections ?
>>>
>>> Thanks,
>>> Cham
>>>
>>> [1]
>>> https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com&ctz=America/Los_Angeles
>>>
>>