Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Charles Chen
+1 Note that we need to temporarily revert https://github.com/apache/beam/pull/6683 before the release branch cut per the discussion at https://lists.apache.org/thread.html/78fe33dc41b04886f5355d66d50359265bfa2985580bb70f79c53545@%3Cdev.beam.apache.org%3E On Thu, Nov 15, 2018 at 9:18 PM Tim

A new Beam Runner on Apache Nemo

2018-11-15 Thread 송원욱
Hello all! I'm a member of the Apache Nemo community, another Apache project for processing big data focusing on easy-to-use, flexible optimizations for various deployment environments. More information can be seen on our website . We've been building the system for quite

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Tim
Thanks Cham +1 > On 16 Nov 2018, at 05:30, Thomas Weise wrote: > > +1 > > >> On Thu, Nov 15, 2018 at 4:34 PM Ahmet Altay wrote: >> +1 Thank you. >> >>> On Thu, Nov 15, 2018 at 4:22 PM, Kenneth Knowles wrote: >>> SGTM. Thanks for keeping track of the schedule. >>> >>> Kenn >>> On

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Thomas Weise
+1 On Thu, Nov 15, 2018 at 4:34 PM Ahmet Altay wrote: > +1 Thank you. > > On Thu, Nov 15, 2018 at 4:22 PM, Kenneth Knowles wrote: > >> SGTM. Thanks for keeping track of the schedule. >> >> Kenn >> >> On Thu, Nov 15, 2018 at 1:59 PM Chamikara Jayalath >> wrote: >> >>> Hi All, >>> >>>

Re: [VOTE] Release Vendored gRPC 1.13.1 and Guava 20.0, release candidate #1

2018-11-15 Thread Thomas Weise
Thanks for driving this. Did we reach a conclusion regarding publishing relocated source artifacts? Debugging would be painful without (unless manually installed in the local repo). On Thu, Nov 15, 2018 at 6:05 PM Lukasz Cwik wrote: > Please review and vote on the release candidate #1 for the

Re: [VOTE] Mark 2.7.0 branch as a long term support (LTS) branch

2018-11-15 Thread Ahmet Altay
Thank you all for voting. This vote was open for 7 days, let's wrap it up. There were 8 +1, 1 +0, and no -1 votes. - I added a new version 2.7.1 for tracking anything that could be used for tracking whatever we would like to consider for backporting (

Re: [DISCUSS] Publish vendored dependencies independently

2018-11-15 Thread Lukasz Cwik
I have created the artifacts and sent out a vote thread. On Thu, Nov 15, 2018 at 5:23 PM Lukasz Cwik wrote: > On Thu, Nov 15, 2018 at 11:47 AM Thomas Weise wrote: > >> In case any of this affects how artifacts are published in general, >> please make sure that publishing to 3rd party repo

[VOTE] Release Vendored gRPC 1.13.1 and Guava 20.0, release candidate #1

2018-11-15 Thread Lukasz Cwik
Please review and vote on the release candidate #1 for the vendored artifacts gRPC 1.13.1 and Guava 20.0: [ ] +1, Approve the release [ ] -1, Do not approve the release (please provide specific comments) The creation of these artifacts are the outcome of the discussion about vendoring[1]. The

Re: [DISCUSS] Publish vendored dependencies independently

2018-11-15 Thread Lukasz Cwik
On Thu, Nov 15, 2018 at 11:47 AM Thomas Weise wrote: > In case any of this affects how artifacts are published in general, please > make sure that publishing to 3rd party repo continues to work. > > For example: ./gradlew :beam-runners-flink_2.11-job-server:publish > -PisRelease -PnoSigning

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Ahmet Altay
+1 Thank you. On Thu, Nov 15, 2018 at 4:22 PM, Kenneth Knowles wrote: > SGTM. Thanks for keeping track of the schedule. > > Kenn > > On Thu, Nov 15, 2018 at 1:59 PM Chamikara Jayalath > wrote: > >> Hi All, >> >> According to the release calendar [1] branch cut date for Beam 2.9.0 >> release is

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Kenneth Knowles
SGTM. Thanks for keeping track of the schedule. Kenn On Thu, Nov 15, 2018 at 1:59 PM Chamikara Jayalath wrote: > Hi All, > > According to the release calendar [1] branch cut date for Beam 2.9.0 > release is 11/21/2018. Since previous release branch was cut close to the > respective calendar

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Andrew Pilloud
I've spent most of the day getting beam working on Calcite 1.18 and now I'm just starting to work on making your Joda UDF changes compatible with Calcite's code generation, so I haven't gotten very far into investigating how we might handle this yet. Calcite has multiple aggregate function

Re: Need help regarding memory leak issue

2018-11-15 Thread Ruoyun Huang
trying to understand the situation you are having. By saying 'kills the appllication', is that a leak in the application itself, or the workers being the root cause? Also are you running ML models inside Python SDK DoFn's? Then I suppose it is running some predictions rather than model

Apache Beam Newsletter - November 2018

2018-11-15 Thread Rose Nguyen
[image: Beam.png] November 2018 | Newsletter What’s been done Beam Community Metrics (by: Mikhail Gryzykhin, Udi Meiri, Huygaa Batsaikhan) - To help track project health status, added dashboarding platform. - Initial dashboards were created that aim at tracking pre- and

Re: Please do not merge Python PRs

2018-11-15 Thread Udi Meiri
All clear, Python tests are reporting errors correctly again. On Wed, Nov 14, 2018 at 5:57 PM Udi Meiri wrote: > https://github.com/apache/beam/pull/7048 is the rollback PR > > On Wed, Nov 14, 2018 at 5:28 PM Ahmet Altay wrote: > >> Thank you Udi. Could you send a rollback PR? >> >> I believe

[PROPOSAL] Prepare Beam 2.9.0 release

2018-11-15 Thread Chamikara Jayalath
Hi All, According to the release calendar [1] branch cut date for Beam 2.9.0 release is 11/21/2018. Since previous release branch was cut close to the respective calendar date I'd like to propose cutting release branch for 2.9.0 on 11/21/2018. Next week is Thanksgiving holiday in US and possibly

Re: Design review for supporting AutoValue Coders and conversions to Row

2018-11-15 Thread Reuven Lax
I would hope so if possible. On Fri, Nov 16, 2018, 4:36 AM Kenneth Knowles Just some low-level detail: If there is no @DefaultSchema annotation but > it is an @AutoValue class, can schema inference go ahead with the > AutoValueSchema? Then the user doesn't have to do anything. > > Kenn > > On

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Rui Wang
For the not working SESSION_END(), I had an investigation on it: https://issues.apache.org/jira/browse/BEAM-5799 https://issues.apache.org/jira/browse/CALCITE-2645 According to the reply in the Calcite JIRA, there might be some other way to implement SESSION_END. I haven't looked into it though.

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Mingmin Xu
1. Window start/end: Actually this is already provided in other ways and the window in the SQL environment is unused and just waiting to be deleted. So you can still access TUMBLE_START, etc. This is well-defined as a part of the row so there's no semantic problem, but I think it should already

Re: Design review for supporting AutoValue Coders and conversions to Row

2018-11-15 Thread Anton Kedin
One reason is that @AutoValue is not guaranteed to be retained at runtime: https://github.com/google/auto/blob/master/value/src/main/java/com/google/auto/value/AutoValue.java#L44 On Thu, Nov 15, 2018 at 11:36 AM Kenneth Knowles wrote: > Just some low-level detail: If there is no @DefaultSchema

Re: [DISCUSS] Publish vendored dependencies independently

2018-11-15 Thread Thomas Weise
In case any of this affects how artifacts are published in general, please make sure that publishing to 3rd party repo continues to work. For example: ./gradlew :beam-runners-flink_2.11-job-server:publish -PisRelease -PnoSigning -PdistMgmtSnapshotsUrl= https://mycustomrepo/libs-release Thanks,

Re: Design review for supporting AutoValue Coders and conversions to Row

2018-11-15 Thread Kenneth Knowles
Just some low-level detail: If there is no @DefaultSchema annotation but it is an @AutoValue class, can schema inference go ahead with the AutoValueSchema? Then the user doesn't have to do anything. Kenn On Wed, Nov 14, 2018 at 6:14 AM Reuven Lax wrote: > We already have a framework for

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Kenneth Knowles
>From https://issues.apache.org/jira/browse/BEAM-5204 it seems like what you most care about is to have joins that trigger more than once per window. To accomplish it you hope to build an "escape hatch" from SQL/relational semantics to specialized Beam SQL semantics. It could make sense with

Re: Migrating Beam SQL to Calcite's code generation

2018-11-15 Thread Mingmin Xu
Raise this thread. Seems there're more changes in the backend on how a FUNCTION is executed in the backend, as noticed in #6996 : 1. BeamSqlExpression and BeamSqlExpressionExecutor are removed; 2. BeamSqlExpressionEnvironment are removed; Then, 1. for

Re: [DISCUSS] SplittableDoFn Java SDK User Facing API

2018-11-15 Thread Ismaël Mejía
Some late comments, and my pre excuses if some questions look silly, but the last documents were a lot of info that I have not yet fully digested. I have some questions about the ‘new’ Backlog concept following a quick look at the PR https://github.com/apache/beam/pull/6969/files 1. Is the

Re: Python profiling

2018-11-15 Thread Thomas Weise
Hi Robert, This is great. It should be added to our Python documentation because users will like need this! After I installed gprof2dot I'm still prompted to install: "Please install gprof2dot and dot for profile renderings." Also is there a way to run a pipeline unmodified with fn_api_runner?

Wiki edit access

2018-11-15 Thread Wout Scheepers
Can anyone give me edit access for the wiki? Thanks, Wout

Re: How to use "PortableRunner" in Python SDK?

2018-11-15 Thread Maximilian Michels
1) The default behavior, where PortableRunner starts a flink server. It is confusing to new users It does that only if no JobServer endpoint is specified. AFAIK there a problems with the bootstrapping, it can definitely be improved. 2) All the related docs and inline comments. Similarly, it

Re: Bigquery streaming TableRow size limit

2018-11-15 Thread Wout Scheepers
Thanks for your thoughts. Also, I’m doing something similar when streaming data into partitioned tables. From [1]: “ When the data is streamed, data between 7 days in the past and 3 days in the future is placed in the streaming buffer, and then it is extracted to the corresponding partitions.”