Consider Cloudpickle instead of dill for Python pickling

2021-04-29 Thread Stephan Hoyer
cloudpickle [1] and dill [2] are two Python packages that implement
extensions of Python's pickle protocol for arbitrary objects. Beam
currently uses dill, but I'm wondering if we could consider additionally or
alternatively use cloudpickle instead.

Overall, cloudpickle seems to be a more popular choice for extended pickle
support in distributing computing in Python, e.g., it's used by Spark, Dask
and joblib.

One of the major differences between cloudpickle and dill is how they
handle pickling global variables (such as Python modules) that are referred
to by a function:
- Dill doesn't serialize globals. If you want to save globals, you need to
call dill.dump_session(). This is what the "save_main_session" flag does in
Beam.
- Cloudpickle takes a different approach. It introspects which global
variables are used by a function, and creates a closure around the
serialized function that only contains these variables.

The cloudpickle approach results in larger serialized functions, but it's
also much more robust, because the required globals are included by
default. In contrast, with dill, one either needs to save *all *globals or
none. This is repeated pain-point for Beam Python users [3]:
- Saving all globals can be overly aggressive, particularly in notebooks
where users may have incidentally created large objects.
- Alternatively, users can avoid using global variables entirely, but this
makes defining ad-hoc pipelines very awkward. Mapped over functions need to
be imported from other modules, or need to have their imports defined
inside the function itself.

I'd love to see an option to use cloudpickle in Beam instead of dill, and
to consider switching over entirely. Cloudpickle would allow Beam users to
write readable code in the way they expect, without needing to worry about
the confusing and potentially problematic "save_main_session" flag.

Any thoughts?

Cheers,
Stephan

[1] https://github.com/cloudpipe/cloudpickle
[2] https://github.com/uqfoundation/dill
[3]
https://cloud.google.com/dataflow/docs/resources/faq#how_do_i_handle_nameerrors


Re: Customizable Artifacts to Stage in Java Portable Runner

2021-04-29 Thread Ke Wu
Got you. We are definitely interested in java worker pool to support Samza 
runner use case, and I think we could help implement on it if no one is 
currently working on it.

Comparing with what python offers, what I see that are missing are:

1. Main class/method to start ExternalWorkerService independently 
2. Worker pool mode support in JDK Docker container in boot.go

Is there anything else I missed?

Best,
Ke

> On Apr 29, 2021, at 12:54 PM, Kyle Weaver  wrote:
> 
> Thanks for the info. In order to use supported remote file systems, does it 
> mean it needs to be passed in as FILE_ARTIFACT_URN since neither 
> ArtifactRetrievalService#URL_ARTIFACT_URN = "beam:artifact:type:url:v1” nor 
> ArtifactRetrievalService#STAGING_TO_ARTIFACT_URN = 
> "beam:artifact:role:staging_to:v1” seems to be supported in getArtifact()?
> 
> Yes.
> 
> By the way, it seems the Python implementation of artifact_service does 
> handle URLs [1] - though it might not support them at every level of the 
> stack [2].
>  
> 
> On the other side, under circumstances, such as EXTERNAL environment type 
> with ExternalWorkerService, where artifacts are already available, what is 
> the expected usage to disable artifact staging phase in portable pipeline?
> 
> I think you can just set --filesToStage to empty.
>  
> 
> In addition, I noticed that the python counterpart 
> worker_pool_main#BeamFnExternalWorkerPoolServicer 
> 
>  does invoke artifact staging service to get artifacts from artifact endpoint 
> specified in StartWorkerRequest but not in the java ExternalWorkerService. Is 
> this discrepancy expected since java worker pool process does not likely want 
> to start the worker with different classpath/classloader?
> 
> 
> It looks like ExternalWorkerService is only used for LOOPBACK mode in Java, 
> so I assume artifact staging/retrieval would be redundant. Whereas in Python, 
> the worker pool you linked to is started independently of job submission. But 
> there's no inherent reason it has to be that way. For example, someday we may 
> want to implement a Java worker pool [3].
> 
> [1] 
> https://github.com/apache/beam/blob/e0136ffc176d157d0928e7d501bca4daca3160a8/sdks/python/apache_beam/runners/portability/artifact_service.py#L81-L85
>  
> 
> [2] https://issues.apache.org/jira/browse/BEAM-11275 
> 
> [3] https://issues.apache.org/jira/browse/BEAM-8137 
> 
> On Wed, Apr 28, 2021 at 6:36 PM Ke Wu  > wrote:
> Thank you Kyle for the prompt response.
> 
> > Yeah, that looks like a bug.
> 
> Created BEAM-12251  to 
> track the issue.
> 
> > Files can use any of Beam's supported remote file systems (GCS, S3, Azure 
> > Blobstore, HDFS). But arbitrary URLs are not supported.
> 
> Thanks for the info. In order to use supported remote file systems, does it 
> mean it needs to be passed in as FILE_ARTIFACT_URN since neither 
> ArtifactRetrievalService#URL_ARTIFACT_URN = "beam:artifact:type:url:v1” nor 
> ArtifactRetrievalService#STAGING_TO_ARTIFACT_URN = 
> "beam:artifact:role:staging_to:v1” seems to be supported in getArtifact()?
> 
> On the other side, under circumstances, such as EXTERNAL environment type 
> with ExternalWorkerService, where artifacts are already available, what is 
> the expected usage to disable artifact staging phase in portable pipeline?
> 
> In addition, I noticed that the python counterpart 
> worker_pool_main#BeamFnExternalWorkerPoolServicer 
> 
>  does invoke artifact staging service to get artifacts from artifact endpoint 
> specified in StartWorkerRequest but not in the java ExternalWorkerService. Is 
> this discrepancy expected since java worker pool process does not likely want 
> to start the worker with different classpath/classloader?
> 
> Best,
> Ke
> 
> 
>> On Apr 28, 2021, at 5:55 PM, Kyle Weaver > > wrote:
>> 
>> > I am expecting FileStagingOptions#setFilesToStage in 
>> > PortablePipelineOptions 
>> > 
>> >  is the way to customize artifacts to be staged and resolved in portable 
>> > pipeline, however, it looks like that PortableRunner 
>> > 
>> >  does not add preconfigured files to `filesToStageBuilder` which is used 
>> > in the final options to prepare the job. 

[DISCUSS] Warn when KafkaIO is used as a bounded source

2021-04-29 Thread Brian Hulette
Our oldest open P1 issue is BEAM-6466 - "KafkaIO doesn't commit offsets
while being used as bounded source" [1]. I'm not sure this is an actual
issue since KafkaIO doesn't seem to officially support this use-case. The
relevant parameters indicate they are "mainly used for tests and demo
applications" [2], and BEAM-2185 - "KafkaIO bounded source" [3] is still
open.

I think we should close out BEAM-6466 by more clearly indicating that
withMaxReadTime() and withMaxRecords() are experimental, and/or logging a
warning when they are used.

I'm happy to make such a change, but I wanted to check if there are any
objections to this first.

Thanks,
Brian

[1] https://issues.apache.org/jira/browse/BEAM-6466
[2]
https://github.com/apache/beam/blob/3d4db26cfa4ace0a0f2fbb602f422fe30670c35f/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java#L960
[3] https://issues.apache.org/jira/browse/BEAM-2185


Flaky test issue report

2021-04-29 Thread Beam Jira Bot
This is your daily summary of Beam's current flaky tests. These are P1 issues 
because they have a major negative impact on the community and make it hard to 
determine the quality of the software.

BEAM-12250: Java ValidatesRunner Postcommits timing out 
(https://issues.apache.org/jira/browse/BEAM-12250)
BEAM-12200: SamzaStoreStateInternalsTest is flaky 
(https://issues.apache.org/jira/browse/BEAM-12200)
BEAM-12163: Python GHA PreCommits flake with grpc.FutureTimeoutError on SDK 
harness startup (https://issues.apache.org/jira/browse/BEAM-12163)
BEAM-12061: beam_PostCommit_SQL failing on 
KafkaTableProviderIT.testFakeNested 
(https://issues.apache.org/jira/browse/BEAM-12061)
BEAM-12020: :sdks:java:container:java8:docker failing missing licenses 
(https://issues.apache.org/jira/browse/BEAM-12020)
BEAM-12019: 
apache_beam.runners.portability.flink_runner_test.FlinkRunnerTestOptimized.test_flink_metrics
 is flaky (https://issues.apache.org/jira/browse/BEAM-12019)
BEAM-11837: Null checking causes build flakes: "Memory constraints are 
impeding performance" (https://issues.apache.org/jira/browse/BEAM-11837)
BEAM-11792: Python precommit failed (flaked?) installing package  
(https://issues.apache.org/jira/browse/BEAM-11792)
BEAM-11666: 
apache_beam.runners.interactive.recording_manager_test.RecordingManagerTest.test_basic_execution
 is flaky (https://issues.apache.org/jira/browse/BEAM-11666)
BEAM-11662: elasticsearch tests failing 
(https://issues.apache.org/jira/browse/BEAM-11662)
BEAM-11661: hdfsIntegrationTest flake: network not found (py38 postcommit) 
(https://issues.apache.org/jira/browse/BEAM-11661)
BEAM-11646: beam_PostCommit_XVR_Spark failing 
(https://issues.apache.org/jira/browse/BEAM-11646)
BEAM-11645: beam_PostCommit_XVR_Flink failing 
(https://issues.apache.org/jira/browse/BEAM-11645)
BEAM-11541: testTeardownCalledAfterExceptionInProcessElement flakes on 
direct runner. (https://issues.apache.org/jira/browse/BEAM-11541)
BEAM-11540: Linter sometimes flakes on apache_beam.dataframe.frames_test 
(https://issues.apache.org/jira/browse/BEAM-11540)
BEAM-10995: Java + Universal Local Runner: 
WindowingTest.testWindowPreservation fails 
(https://issues.apache.org/jira/browse/BEAM-10995)
BEAM-10987: stager_test.py::StagerTest::test_with_main_session flaky on 
windows py3.6,3.7 (https://issues.apache.org/jira/browse/BEAM-10987)
BEAM-10968: flaky test: 
org.apache.beam.sdk.metrics.MetricsTest$AttemptedMetricTests.testAttemptedDistributionMetrics
 (https://issues.apache.org/jira/browse/BEAM-10968)
BEAM-10955: Flink Java Runner test flake: Could not find Flink job  
(https://issues.apache.org/jira/browse/BEAM-10955)
BEAM-10923: Python requirements installation in docker container is flaky 
(https://issues.apache.org/jira/browse/BEAM-10923)
BEAM-10899: test_FhirIO_exportFhirResourcesGcs flake with OOM 
(https://issues.apache.org/jira/browse/BEAM-10899)
BEAM-10866: PortableRunnerTestWithSubprocesses.test_register_finalizations 
flaky on macOS (https://issues.apache.org/jira/browse/BEAM-10866)
BEAM-10763: Spotless flake (NullPointerException) 
(https://issues.apache.org/jira/browse/BEAM-10763)
BEAM-10590: BigQueryQueryToTableIT flaky: test_big_query_new_types 
(https://issues.apache.org/jira/browse/BEAM-10590)
BEAM-10519: 
MultipleInputsAndOutputTests.testParDoWithSideInputsIsCumulative flaky on Samza 
(https://issues.apache.org/jira/browse/BEAM-10519)
BEAM-10504: Failure / flake in ElasticSearchIOTest > 
testWriteFullAddressing and testWriteWithIndexFn 
(https://issues.apache.org/jira/browse/BEAM-10504)
BEAM-10501: CheckGrafanaStalenessAlerts and PingGrafanaHttpApi fail with 
Connection refused (https://issues.apache.org/jira/browse/BEAM-10501)
BEAM-10485: Failure / flake: ElasticsearchIOTest > testWriteWithIndexFn 
(https://issues.apache.org/jira/browse/BEAM-10485)
BEAM-10272: Failure in CassandraIOTest init: cannot create cluster due to 
netty link error (https://issues.apache.org/jira/browse/BEAM-10272)
BEAM-9649: beam_python_mongoio_load_test started failing due to mismatched 
results (https://issues.apache.org/jira/browse/BEAM-9649)
BEAM-9392: TestStream tests are all flaky 
(https://issues.apache.org/jira/browse/BEAM-9392)
BEAM-9232: BigQueryWriteIntegrationTests is flaky coercing to Unicode 
(https://issues.apache.org/jira/browse/BEAM-9232)
BEAM-9119: 
apache_beam.runners.portability.fn_api_runner_test.FnApiRunnerTest[...].test_large_elements
 is flaky (https://issues.apache.org/jira/browse/BEAM-9119)
BEAM-8879: IOError flake in PortableRunnerTest 
(https://issues.apache.org/jira/browse/BEAM-8879)
BEAM-8101: Flakes in 
ParDoLifecycleTest.testTeardownCalledAfterExceptionInStartBundleStateful for 
Direct, Spark, Flink (https://issues.apache.org/jira/browse/BEAM-8101)
BEAM-8035: [beam_PreCommit_Java_Phrase] 
[WatchTest.testMultiplePollsWithManyResults]  Flake: 

P1 issues report

2021-04-29 Thread Beam Jira Bot
This is your daily summary of Beam's current P1 issues, not including flaky 
tests.

See https://beam.apache.org/contribute/jira-priorities/#p1-critical for the 
meaning and expectations around P1 issues.

BEAM-12256: PubsubIO.readAvroGenericRecord creates SchemaCoder that fails 
to decode some Avro logical types 
(https://issues.apache.org/jira/browse/BEAM-12256)
BEAM-12242: An incorrect model pipeline is generated when PubSub is 
followed by a x-lang transform for streaming 
(https://issues.apache.org/jira/browse/BEAM-12242)
BEAM-12231: beam_PostRelease_NightlySnapshot failing 
(https://issues.apache.org/jira/browse/BEAM-12231)
BEAM-12229: WindmillStateCache has a 0% hit rate in 2.29 
(https://issues.apache.org/jira/browse/BEAM-12229)
BEAM-1: Dataflow side input translation "Unknown producer for value" 
(https://issues.apache.org/jira/browse/BEAM-1)
BEAM-12195: Flink Runner 1.11 uses old Scala-Version 
(https://issues.apache.org/jira/browse/BEAM-12195)
BEAM-11959: Python Beam SDK Harness hangs when installing pip packages 
(https://issues.apache.org/jira/browse/BEAM-11959)
BEAM-11906: No trigger early repeatedly for session windows 
(https://issues.apache.org/jira/browse/BEAM-11906)
BEAM-11875: XmlIO.Read does not handle XML encoding per spec 
(https://issues.apache.org/jira/browse/BEAM-11875)
BEAM-11828: JmsIO is not acknowledging messages correctly 
(https://issues.apache.org/jira/browse/BEAM-11828)
BEAM-11755: Cross-language consistency (RequiresStableInputs) is quietly 
broken (at least on portable flink runner) 
(https://issues.apache.org/jira/browse/BEAM-11755)
BEAM-11578: `dataflow_metrics` (python) fails with TypeError (when int 
overflowing?) (https://issues.apache.org/jira/browse/BEAM-11578)
BEAM-11576: Go ValidatesRunner failure: TestFlattenDup on Dataflow Runner 
(https://issues.apache.org/jira/browse/BEAM-11576)
BEAM-11434: Expose Spanner admin/batch clients in Spanner Accessor 
(https://issues.apache.org/jira/browse/BEAM-11434)
BEAM-11227: Upgrade beam-vendor-grpc-1_26_0-0.3 to fix CVE-2020-27216 
(https://issues.apache.org/jira/browse/BEAM-11227)
BEAM-11148: Kafka commitOffsetsInFinalize OOM on Flink 
(https://issues.apache.org/jira/browse/BEAM-11148)
BEAM-11017: Timer with dataflow runner can be set multiple times (dataflow 
runner) (https://issues.apache.org/jira/browse/BEAM-11017)
BEAM-10861: Adds URNs and payloads to PubSub transforms 
(https://issues.apache.org/jira/browse/BEAM-10861)
BEAM-10617: python CombineGlobally().with_fanout() cause duplicate combine 
results for sliding windows (https://issues.apache.org/jira/browse/BEAM-10617)
BEAM-10569: SpannerIO tests don't actually assert anything. 
(https://issues.apache.org/jira/browse/BEAM-10569)
BEAM-10288: Quickstart documents are out of date 
(https://issues.apache.org/jira/browse/BEAM-10288)
BEAM-10244: Populate requirements cache fails on poetry-based packages 
(https://issues.apache.org/jira/browse/BEAM-10244)
BEAM-10100: FileIO writeDynamic with AvroIO.sink not writing all data 
(https://issues.apache.org/jira/browse/BEAM-10100)
BEAM-9564: Remove insecure ssl options from MongoDBIO 
(https://issues.apache.org/jira/browse/BEAM-9564)
BEAM-9455: Environment-sensitive provisioning for Dataflow 
(https://issues.apache.org/jira/browse/BEAM-9455)
BEAM-9293: Python direct runner doesn't emit empty pane when it should 
(https://issues.apache.org/jira/browse/BEAM-9293)
BEAM-8986: SortValues may not work correct for numerical types 
(https://issues.apache.org/jira/browse/BEAM-8986)
BEAM-8985: SortValues should fail if SecondaryKey coder is not 
deterministic (https://issues.apache.org/jira/browse/BEAM-8985)
BEAM-8407: [SQL] Some Hive tests throw NullPointerException, but get marked 
as passing (Direct Runner) (https://issues.apache.org/jira/browse/BEAM-8407)
BEAM-7717: PubsubIO watermark tracking hovers near start of epoch 
(https://issues.apache.org/jira/browse/BEAM-7717)
BEAM-7716: PubsubIO returns empty message bodies for all messages read 
(https://issues.apache.org/jira/browse/BEAM-7716)
BEAM-7195: BigQuery - 404 errors for 'table not found' when using dynamic 
destinations - sometimes, new table fails to get created 
(https://issues.apache.org/jira/browse/BEAM-7195)
BEAM-6839: User reports protobuf ClassChangeError running against 2.6.0 or 
above (https://issues.apache.org/jira/browse/BEAM-6839)
BEAM-6466: KafkaIO doesn't commit offsets while being used as bounded 
source (https://issues.apache.org/jira/browse/BEAM-6466)


Re: Customizable Artifacts to Stage in Java Portable Runner

2021-04-29 Thread Kyle Weaver
>
> Thanks for the info. In order to use supported remote file systems, does
> it mean it needs to be passed in as FILE_ARTIFACT_URN since neither
> *ArtifactRetrievalService#URL_ARTIFACT_URN = "beam:artifact:type:url:v1” *
> nor *ArtifactRetrievalService#STAGING_TO_ARTIFACT_URN =
> "beam:artifact:role:staging_to:v1” *seems to be supported in
> getArtifact()?
>

Yes.

By the way, it seems the Python implementation of artifact_service
*does *handle
URLs [1] - though it might not support them at every level of the stack [2].


>
> On the other side, under circumstances, such as EXTERNAL environment type
> with ExternalWorkerService, where artifacts are already available, what is
> the expected usage to disable artifact staging phase in portable pipeline?
>

I think you can just set --filesToStage to empty.


>
> In addition, I noticed that the python counterpart
> worker_pool_main#BeamFnExternalWorkerPoolServicer
> 
>  does
> invoke artifact staging service to get artifacts from artifact endpoint
> specified in StartWorkerRequest but not in the java ExternalWorkerService.
> Is this discrepancy expected since java worker pool process does not likely
> want to start the worker with different classpath/classloader?
>
>
It looks like ExternalWorkerService is only used for LOOPBACK mode in Java,
so I assume artifact staging/retrieval would be redundant. Whereas in
Python, the worker pool you linked to is started independently of job
submission. But there's no inherent reason it has to be that way. For
example, someday we may want to implement a Java worker pool [3].

[1]
https://github.com/apache/beam/blob/e0136ffc176d157d0928e7d501bca4daca3160a8/sdks/python/apache_beam/runners/portability/artifact_service.py#L81-L85
[2] https://issues.apache.org/jira/browse/BEAM-11275
[3] https://issues.apache.org/jira/browse/BEAM-8137

On Wed, Apr 28, 2021 at 6:36 PM Ke Wu  wrote:

> Thank you Kyle for the prompt response.
>
> > Yeah, that looks like a bug.
>
> Created BEAM-12251  to
> track the issue.
>
> > Files can use any of Beam's supported remote file systems (GCS, S3,
> Azure Blobstore, HDFS). But arbitrary URLs are not supported.
>
> Thanks for the info. In order to use supported remote file systems, does
> it mean it needs to be passed in as FILE_ARTIFACT_URN since neither 
> *ArtifactRetrievalService#URL_ARTIFACT_URN = "beam:artifact:type:url:v1”
> *nor *ArtifactRetrievalService#STAGING_TO_ARTIFACT_URN =
> "beam:artifact:role:staging_to:v1” *seems to be supported in
> getArtifact()?
>
> On the other side, under circumstances, such as EXTERNAL environment type
> with ExternalWorkerService, where artifacts are already available, what is
> the expected usage to disable artifact staging phase in portable pipeline?
>
> In addition, I noticed that the python counterpart
> worker_pool_main#BeamFnExternalWorkerPoolServicer
> 
>  does
> invoke artifact staging service to get artifacts from artifact endpoint
> specified in StartWorkerRequest but not in the java ExternalWorkerService.
> Is this discrepancy expected since java worker pool process does not likely
> want to start the worker with different classpath/classloader?
>
> Best,
> Ke
>
>
> On Apr 28, 2021, at 5:55 PM, Kyle Weaver  wrote:
>
> > I am expecting FileStagingOptions#setFilesToStage in
> PortablePipelineOptions
> 
>  is
> the way to customize artifacts to be staged and resolved in portable
> pipeline, however, it looks like that PortableRunner
> 
>  does
> not add preconfigured files to `filesToStageBuilder` which is used in the
> final options to prepare the job. Is this the expected behavior or maybe a
> bug?
>
> Yeah, that looks like a bug.
>
> > In addition, do we support specifying an URL in
> PortablePipelineOptions#filesToStage so that ArtifactRetrievalService can
> retrieve artifacts from a remote address instead of default from JobServer,
> which got artifacts from SDK Client. I am asking because I noticed
>
> Files can use any of Beam's supported remote file systems (GCS, S3, Azure
> Blobstore, HDFS). But arbitrary URLs are not supported.
>
> On Wed, Apr 28, 2021 at 5:44 PM Ke Wu  wrote:
>
>> Hello All,
>>
>> I am expecting FileStagingOptions#setFilesToStage in
>> PortablePipelineOptions
>> 
>>  is
>> the way to customize artifacts to be staged and resolved in portable
>> pipeline, 

Timer.withOutputTimestamp().offset().setRelative seems unusable with event time

2021-04-29 Thread Jan Lukavský

Hi,

I have come across a bug with timer output timestamp - when using event 
time and relative timers, setting the timer can arbitrarily throw 
IllegalArgumentException if the firing timestamp (input watermark) is 
ahead of the output timestamp (like .java.lang.IllegalArgumentException: 
Attempted to set an event-time timer with an output timestamp of 
2021-04-29T07:16:19.369Z that is after the timer firing timestamp 
-290308-12-21T19:59:05.225Z). But there is no way to access the firing 
timestamp from user code. This means that the use has to either catch 
the IllegalArgumentException, or not use this construct at all.


Catching the exception should probably not be part of a contract, so we 
should do one of the following:


 a) either throw the exception right away and disable using relative 
timers with output timestamp completely, or


 b) support it correctly

What is the actual reason not to support output timestamps, that are 
ahead of firing timestamp? From my understanding, that should not be an 
issue, because there is TimestampCombiner.EARLIEST on the 
watermarkholdstate that corresponds to the output timestamp. If that is 
correct can we simply remove the check?


 Jan



Re: [PROPOSAL] Preparing for Beam 2.30.0 release

2021-04-29 Thread Heejong Lee
We have 10 open issues for Fix Version 2.30.0:
https://issues.apache.org/jira/browse/BEAM-12242?jql=project%20%3D%20BEAM%20AND%20status%20in%20(Open%2C%20%22In%20Progress%22%2C%20%22In%20Implementation%22%2C%20%22Triage%20Needed%22)%20AND%20fixVersion%20%3D%202.30.0

On Thu, Apr 29, 2021 at 12:30 AM Heejong Lee  wrote:

> FYI, I just cut the 2.30.0 release branch. From now on, late commits for
> 2.30.0 need to be cherry-picked. If you have any late commits, please make
> sure that their Jira issues have the correct Fix Version, 2.30.0.
>
> On Tue, Apr 27, 2021 at 7:52 AM Kenneth Knowles  wrote:
>
>> SGTM. Thanks!
>>
>> On Mon, Apr 26, 2021 at 2:33 PM Heejong Lee  wrote:
>>
>>>
>>>
>>> On Mon, Apr 26, 2021 at 10:24 AM Robert Bradshaw 
>>> wrote:
>>>
 Confirming that the cut date is 4/28/2021 (in two days), right?

>>>
>>> Yes, 2.30.0 branch is scheduled to be cut on April 28.
>>>
>>>

 On Wed, Apr 21, 2021 at 4:41 PM Tomo Suzuki  wrote:
 >
 > Thank you for the preparation!
 >
 > > a few responses that some high priority changes
 >
 > Would you be willing to share the items for visibility?

 There are several PRs in flight (or recently merged) to get
 portability working well with Dataflow for this release.

>>>
>>> We can still cherry-pick them by importance after the branch cut.
>>>
>>>

 >
 > On Wed, Apr 21, 2021 at 7:21 PM Kenneth Knowles 
 wrote:
 > >
 > > Also the 2.29.0 was re-cut.
 > >
 > > Usually a delay in one release should not delay the next release,
 because each release represents a certain quantity of changes. But in this
 case, the actual quantity of changes is affected by the re-cut, too.
 > >
 > > On Wed, Apr 21, 2021 at 4:12 PM Heejong Lee 
 wrote:
 > >>
 > >> Update on the 2.30.0 branch cut schedule:
 > >>
 > >> I'm thinking of delaying the branch cut a week since I've got a
 few responses that some high priority changes are still ongoing.
 > >>
 > >> The new cut date is April 28.
 > >>
 > >>
 > >> On Tue, Apr 20, 2021 at 6:07 PM Ahmet Altay 
 wrote:
 > >>>
 > >>> +1 and thank you!
 > >>>
 > >>> On Tue, Apr 20, 2021 at 4:55 PM Heejong Lee 
 wrote:
 > 
 >  Hi All,
 > 
 >  Beam 2.30.0 release is scheduled to be cut on April 21 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,
 >  Heejong
 > 
 >  [1]
 https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com=America/Los_Angeles
 >
 >
 >
 > --
 > Regards,
 > Tomo

>>>


Re: [PROPOSAL] Preparing for Beam 2.30.0 release

2021-04-29 Thread Heejong Lee
FYI, I just cut the 2.30.0 release branch. From now on, late commits for
2.30.0 need to be cherry-picked. If you have any late commits, please make
sure that their Jira issues have the correct Fix Version, 2.30.0.

On Tue, Apr 27, 2021 at 7:52 AM Kenneth Knowles  wrote:

> SGTM. Thanks!
>
> On Mon, Apr 26, 2021 at 2:33 PM Heejong Lee  wrote:
>
>>
>>
>> On Mon, Apr 26, 2021 at 10:24 AM Robert Bradshaw 
>> wrote:
>>
>>> Confirming that the cut date is 4/28/2021 (in two days), right?
>>>
>>
>> Yes, 2.30.0 branch is scheduled to be cut on April 28.
>>
>>
>>>
>>> On Wed, Apr 21, 2021 at 4:41 PM Tomo Suzuki  wrote:
>>> >
>>> > Thank you for the preparation!
>>> >
>>> > > a few responses that some high priority changes
>>> >
>>> > Would you be willing to share the items for visibility?
>>>
>>> There are several PRs in flight (or recently merged) to get
>>> portability working well with Dataflow for this release.
>>>
>>
>> We can still cherry-pick them by importance after the branch cut.
>>
>>
>>>
>>> >
>>> > On Wed, Apr 21, 2021 at 7:21 PM Kenneth Knowles 
>>> wrote:
>>> > >
>>> > > Also the 2.29.0 was re-cut.
>>> > >
>>> > > Usually a delay in one release should not delay the next release,
>>> because each release represents a certain quantity of changes. But in this
>>> case, the actual quantity of changes is affected by the re-cut, too.
>>> > >
>>> > > On Wed, Apr 21, 2021 at 4:12 PM Heejong Lee 
>>> wrote:
>>> > >>
>>> > >> Update on the 2.30.0 branch cut schedule:
>>> > >>
>>> > >> I'm thinking of delaying the branch cut a week since I've got a few
>>> responses that some high priority changes are still ongoing.
>>> > >>
>>> > >> The new cut date is April 28.
>>> > >>
>>> > >>
>>> > >> On Tue, Apr 20, 2021 at 6:07 PM Ahmet Altay 
>>> wrote:
>>> > >>>
>>> > >>> +1 and thank you!
>>> > >>>
>>> > >>> On Tue, Apr 20, 2021 at 4:55 PM Heejong Lee 
>>> wrote:
>>> > 
>>> >  Hi All,
>>> > 
>>> >  Beam 2.30.0 release is scheduled to be cut on April 21 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,
>>> >  Heejong
>>> > 
>>> >  [1]
>>> https://calendar.google.com/calendar/u/0/embed?src=0p73sl034k80oob7seouani...@group.calendar.google.com=America/Los_Angeles
>>> >
>>> >
>>> >
>>> > --
>>> > Regards,
>>> > Tomo
>>>
>>