Re: Python SDK Arrow Integrations

2019-03-28 Thread Kenneth Knowles
On Thu, Mar 28, 2019 at 12:24 PM Brian Hulette wrote: > > - Presumably there is a pandas counterpart in Java. Is there? Do you > know? > I think there are some dataframe libraries in Java we could look into. I'm > not aware of anything that has the same popularity and arrow integration as >

JDBCIO Connection Pooling

2019-03-28 Thread hgu2hw+2g0aed6fdoszs
Hello, I have recently created a streaming google dataflow program with apache beam using the java SDK. When files land in cloud-storage they fire off pubsub messages with the filename, which I consume and then write to a cloud sql database. Everything works great for the most part. However

Re: Unexpected TestStream behavior when testing stateful DoFn

2019-03-28 Thread Ahmet Altay
I agree it looks like a bug. Filed https://issues.apache.org/jira/browse/BEAM-6934 On Thu, Mar 28, 2019 at 5:18 PM Reuven Lax wrote: > This looks like a bug to me. > > On Thu, Mar 28, 2019 at 2:52 PM Amar Pai wrote: > >> Hi, >> >> I'm running into some unexpected behavior when trying to unit

Re: Python SDK Arrow Integrations

2019-03-28 Thread Ahmet Altay
On Thu, Mar 28, 2019 at 12:24 PM Brian Hulette wrote: > > I think splitting to new transforms rather that adding new options to > existing IO transforms would be simpler for users. I think this would be a > question that could be easier to answer with a PR. > Ok I'll start working on one :) > >

Re: Unexpected TestStream behavior when testing stateful DoFn

2019-03-28 Thread Reuven Lax
This looks like a bug to me. On Thu, Mar 28, 2019 at 2:52 PM Amar Pai wrote: > Hi, > > I'm running into some unexpected behavior when trying to unit test a > stateful DoFn that uses watermark timers as well as bag state. I'm > following the example here: >

Unexpected TestStream behavior when testing stateful DoFn

2019-03-28 Thread Amar Pai
Hi, I'm running into some unexpected behavior when trying to unit test a stateful DoFn that uses watermark timers as well as bag state. I'm following the example here: https://github.com/apache/beam/blob/master/sdks/python/apache_beam/transforms/userstate_test.py#L333 Expected behavior: When

Re: Debugging :beam-sdks-java-io-hadoop-input-format:test

2019-03-28 Thread Mikhail Gryzykhin
I've seen it couple of times already and just got another repro: https://builds.apache.org/job/beam_PreCommit_Java_Commit/5011/consoleFull On Thu, Mar 28, 2019 at 8:55 AM Alexey Romanenko wrote: > Hi Mikhail, > > We had a flaky “HIFIOWithEmbeddedCassandraTest” a while ago and it was > caused by

GSoC Project Proposal

2019-03-28 Thread Tanay Tummalapalli
Hi everyone, I am applying for Google Summer of Code with Apache Beam this year. I have chosen BEAM-6611 - "A Python Sink for BigQuery with File Loads in Streaming" as my project. The draft of my proposal

Re: Python SDK Arrow Integrations

2019-03-28 Thread Chamikara Jayalath
On Wed, Mar 27, 2019 at 9:19 PM Kenneth Knowles wrote: > Thinking about Arrow + Beam SQL + schemas: > > - Obviously many SQL operations could be usefully accelerated by arrow / > columnar. Especially in the analytical realm this is the new normal. For > ETL, perhaps less so. > > - Beam SQL

Re: [PROPOSAL] Standardize Gradle structure in Python SDK

2019-03-28 Thread Mark Liu
Thank you Ahmet. Answer your questions below: > - Could you comment on what kind of parallelization we will gain by this? > In terms of real numbers, how would this affect build and test times? The proposal is based on Gradle parallel execution

Re: Deprecating Avro for fastavro on Python 3

2019-03-28 Thread Ahmet Altay
Hi Ismaël, It is great to hear that Avro is planning to make a release soon. To answer your concerns, fastavro has a set of tests using regular avro files[1] and it also has a large set of users (with 675470 package downloads). This is in addition to it being a py2 & py3 compatible package and

Re: [PROPOSAL] Preparing for Beam 2.12.0 release

2019-03-28 Thread Andrew Pilloud
It seems like there was some confusion around when the branch cut was going to happen. I cut the branch yesterday, but a dozen release blocking fixes went in immediately after. I recut the branch today, one day late, at 1d9daf1

Re: [PROPOSAL] Preparing for Beam 2.12.0 release

2019-03-28 Thread Andrew Pilloud
Yes, that is what I meant. Sorry about mixing up the month! On Thu, Mar 28, 2019 at 9:26 AM Robert Burke wrote: > I'm going to go out on a limb and assume you mean first RC cut on April > 3rd, and the Cherry-pick deadline EoD (PST?) April 2nd. > > On Thu, 28 Mar 2019 at 09:23, Andrew Pilloud

Re: Flink 1.7.2 + Beam 2.11 error: The transform beam:transform:create_view:v1 is currently not supported.

2019-03-28 Thread Lukasz Cwik
+dev On Thu, Mar 28, 2019 at 9:13 AM Kaymak, Tobias wrote: > Hello, > > I just upgraded to Flink 1.7.2 from 1.6.2 with my dev cluster and from > Beam 2.10 to 2.11 and I am seeing this error when starting my pipelines: > > org.apache.flink.client.program.ProgramInvocationException: The main >

Re: Increase Portable SDK Harness share of memory?

2019-03-28 Thread Lukasz Cwik
The intention is that these kinds of hints such as CPU and/or memory should be embedded in the environment specification that is associated with the transforms that need resource hints. The environment spec is woefully ill prepared as it only has a docker URL right now. On Thu, Mar 28, 2019 at

Re: [PROPOSAL] Preparing for Beam 2.12.0 release

2019-03-28 Thread Robert Burke
I'm going to go out on a limb and assume you mean first RC cut on April 3rd, and the Cherry-pick deadline EoD (PST?) April 2nd. On Thu, 28 Mar 2019 at 09:23, Andrew Pilloud wrote: > It seems like there was some confusion around when the branch cut was > going to happen. I cut the branch

Re: Debugging :beam-sdks-java-io-hadoop-input-format:test

2019-03-28 Thread Alexey Romanenko
Hi Mikhail, We had a flaky “HIFIOWithEmbeddedCassandraTest” a while ago and it was caused by issue with launching of embedded Cassandra cluster. Then it was fixed by Etienne Chauchot's PR [1] Though, I don’t see any similar error messages in your Jenkins job log, so, I’m not sure it’s the same

Increase Portable SDK Harness share of memory?

2019-03-28 Thread Robert Burke
A question came over the beam-go slack that I wasn't able to answer, in particular for Dataflow*, is there a way to increase how much of a Portable FnAPI worker is dedicated for the SDK side, vs the Runner side? My assumption is that runners should manage it, and have the Runner Harness side be

Re: Deprecating Avro for fastavro on Python 3

2019-03-28 Thread Ismaël Mejía
Hello, The problem of switching implementations is the risk of losing interoperability, and this is more important than performance. Does fastavro have tests that guarantee that it is fully compatible with Avro’s Java version? (given that it is the de-facto implementation used everywhere). If

Re: NullPointerException - Session windows with Lateness in FlinkRunner

2019-03-28 Thread Maximilian Michels
Hi Rahul, Thanks for providing the detailed report. This looks like a bug rather than a limitation of the Flink Runner. We have integration tests for session windows with the Flink Runner but they seemed to have missed this issue. Let me investigate and get back to you. Tracking issue: