Re: New contributor

2019-03-27 Thread Lukasz Cwik
Welcome. The getting started[1] and contribution guides[2] are most useful. I have also added you as a contributor to the JIRA project. 1: https://beam.apache.org/get-started/beam-overview/ 2: https://beam.apache.org/contribute/ On Wed, Mar 27, 2019 at 9:38 AM Niklas Hansson < niklas.sven.hans...

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 8:

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 > me

Re: Increase Portable SDK Harness share of memory?

2019-04-01 Thread Lukasz Cwik
docker specific one. Using the URN + payload would allow us to have a versioned way to update the environment specifications and deprecate/remove things that are ill defined. On Fri, Mar 29, 2019 at 6:41 PM Kenneth Knowles wrote: > > > On Thu, Mar 28, 2019 at 9:30 AM Lukasz Cwik wrot

Re: [PROPOSAL] Introduce beam-sdks-java gradle project

2019-04-01 Thread Lukasz Cwik
During the gradle migration, we used to have something like: include(":sdks:java:core") include(":sdks:java:extensions:sql") include(":sdks:python") but we discovered the Maven module names that were used during publishing were "core" / "sql" / ... (effectively the directory name) instead of "bea

Re: Increase Portable SDK Harness share of memory?

2019-04-01 Thread Lukasz Cwik
Yes, need to use the new fields everywhere and then deprecate the old fields. On Mon, Apr 1, 2019 at 1:33 PM Kenneth Knowles wrote: > > > On Mon, Apr 1, 2019 at 8:59 AM Lukasz Cwik wrote: > >> To clarify, docker isn't the only environment type we are using. We have

Re: [PROPOSAL] Introduce beam-sdks-java gradle project

2019-04-01 Thread Lukasz Cwik
> since we are using Gradle) or a new artifact that you want to distribute? > > On Mon, Apr 1, 2019 at 10:01 AM Lukasz Cwik wrote: > >> During the gradle migration, we used to have something like: >> >> include(":sdks:java:core") >> include(":sdks:jav

Re: Beam contribution

2019-04-02 Thread Lukasz Cwik
+Ahmed I have added you as a contributor. It seems as though Ahmed had just picked up BEAM-3489 yesterday. Reach out to Ahmed if you would like to help them out with the task. Was TimerReceiverTest failing reliably when performing a parallel build or is it flaky? I have asked Chamikara to take

Re: [PROPOSAL] Introduce beam-sdks-java gradle project

2019-04-02 Thread Lukasz Cwik
> > > > To continue here, lots of cleanup has to be done. We might also > > need to rename folders etc, do better reflect semantic > intentions. > > > > [1] https://github.com/apache/beam/pull/8194 > > > >

Re: [DISCUSS] SplittableDoFn Java SDK User Facing API

2019-04-02 Thread Lukasz Cwik
I was able to update the failing Watch transform in https://github.com/apache/beam/pull/8146 and this has now been merged. On Mon, Mar 18, 2019 at 10:32 AM Lukasz Cwik wrote: > Thanks Kenn, based upon the error message there was a small amount of code > that I missed when updating th

Re: Implementation an S3 file system for python SDK

2019-04-03 Thread Lukasz Cwik
+dev On Wed, Apr 3, 2019 at 2:03 AM Pasan Kamburugamuwa < pasankamburugamu...@gmail.com> wrote: > Hi , > > I have completed a proposal to implementation an S3 file system for python > SDK for the google summer of Code 2019. Please can you guys review this > proposal and if there is any issues wi

Re: [Forked] BEAM-4046 (was [PROPOSAL] Introduce beam-sdks-java gradle project)

2019-04-03 Thread Lukasz Cwik
is already 2 yrs old and iirc was > reported already at least one year earlier - I do not expect a fix here > soon. > > On Tue, Apr 2, 2019 at 7:19 PM Lukasz Cwik wrote: > >> I didn't know that https://github.com/gradle/gradle/issues/847 existed >> but the descriptio

Re: Increase Portable SDK Harness share of memory?

2019-04-03 Thread Lukasz Cwik
Turns out much of the work was completed to populate and consume the urn + payloads. I have deprecated the single "url" field in enviornment with https://github.com/apache/beam/pull/8213 which will allow us to close of BEAM-5433. On Mon, Apr 1, 2019 at 1:48 PM Lukasz Cwik wrote: >

Re: Hazelcast Jet Runner - validation tests

2019-04-04 Thread Lukasz Cwik
The issue with unbounded tests that rely on triggers/late data/early firings/processing time is that these are several sources of non-determinism. The sources make non-deterministic decisions around when to produce data, checkpoint, and resume and runners make non-deterministic decisions around whe

Re: test_split_crazy_sdf broken in python presubmit. 'DataInputOperation' object has no attribute 'index'

2019-04-04 Thread Lukasz Cwik
I have tried running this test at head locally and have not gotten this failure because I also had a different failure related to the .with_complete method not being available. I'm not yet sure whether this is a Jenkins executor issue or an actual code issue. On Thu, Apr 4, 2019 at 9:17 AM Alex Am

Re: test_split_crazy_sdf broken in python presubmit. 'DataInputOperation' object has no attribute 'index'

2019-04-04 Thread Lukasz Cwik
I think its a Jenkins executor issue because https://github.com/apache/beam/pull/8217 passed its test just now. On Thu, Apr 4, 2019 at 10:02 AM Lukasz Cwik wrote: > I have tried running this test at head locally and have not gotten this > failure because I also had a different failure r

Re: [DISCUSS] change the encoding scheme of Python StrUtf8Coder

2019-04-04 Thread Lukasz Cwik
standard_coders.yaml[1] is where we are currently defining these formats. Unfortunately the Python SDK has its own copy[2]. Here is an example PR[3] that adds the "beam:coder:double:v1" as tests to the Java and Python SDKs to ensure interoperability. Robert Burke, does the Go SDK have a test wher

Re: [DISCUSS] change the encoding scheme of Python StrUtf8Coder

2019-04-04 Thread Lukasz Cwik
On Thu, Apr 4, 2019 at 1:15 PM Chamikara Jayalath wrote: > > > On Thu, Apr 4, 2019 at 12:15 PM Lukasz Cwik wrote: > >> standard_coders.yaml[1] is where we are currently defining these formats. >> Unfortunately the Python SDK has its own copy[2]. >> > > Ah gre

Re: Beam contribution

2019-04-04 Thread Lukasz Cwik
TimerReceiverTest tests fail reliably for me. > > > > > > On Tue, 2 Apr 2019 at 23:53, Lukasz Cwik wrote: > >> +Ahmed >> >> I have added you as a contributor. >> >> It seems as though Ahmed had just picked up BEAM-3489 yesterday. Reach >>

Re: Beam contribution

2019-04-05 Thread Lukasz Cwik
or different reasons. Getting timeout in test setup. Unsure, > might indicate some grpc port/server startup issue because previous run did > not do proper cleanup. > > best, > > michel > > [1] > https://github.com/apache/beam/blob/master/sdks/java/harness/src/main/java/org

Re: [DISCUSS] change the encoding scheme of Python StrUtf8Coder

2019-04-05 Thread Lukasz Cwik
>> >> >> Seems like /model is a good place for this if we don't want to >> invent a new language-independent hierarchy. >> >> >> >> >> >> Kenn >> >> >> >> >> >> >

Re: [DISCUSS] change the encoding scheme of Python StrUtf8Coder

2019-04-05 Thread Lukasz Cwik
Also, as for the backwards compatibility discussion, I don't believe non-portable jobs will be able to be upgraded to portable jobs and hence may be a good time to make upgrade incompatible coder changes at that point in time. On Fri, Apr 5, 2019 at 1:44 PM Lukasz Cwik wrote: > Robert,

Re: Beam contribution

2019-04-05 Thread Lukasz Cwik
-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/logging/DataflowWorkerLoggingInitializerTest.java#L81 > > On Fri, Apr 5, 2019 at 10:12 PM Lukasz Cwik wrote: > >> We replace System.out/err to capture user logs and forward the logs for >> the Dataflow

Re: Beam contribution

2019-04-05 Thread Lukasz Cwik
Filed https://issues.apache.org/jira/browse/BEAM-7016 with the details. On Fri, Apr 5, 2019 at 1:47 PM Lukasz Cwik wrote: > Yes, it seems like the reset() method resets System.out even if it never > was initialized. Seems like a simple fix to have reset() be safe to call at > all tim

Re: Implementation an S3 file system for python SDK - Updated

2019-04-08 Thread Lukasz Cwik
1] could probably use SDF-based >> improvements to split when many files are being matched. >> Best >> -P. >> >> On Mon, Apr 8, 2019 at 10:00 AM Alex Amato wrote: >> >>> +Lukasz Cwik , +Boyuan Zhang , +Lara >>> Schmidt >>> >>>

Re: PostCommit tests currently failing

2019-04-09 Thread Lukasz Cwik
The SplittableDoFn failures are because of the changes associated with BEAM-6978. On Tue, Apr 9, 2019 at 5:05 AM Michael Luckey wrote: > Hi, > > looks as if > - beam_PostCommit_Java11_ValidatesRunner_PortabilityApi_Dataflow > - beam_PostCommit_Java_Nexmark_Dataflow > - beam_PostCommit_Java_Valid

Re: PostCommit tests currently failing

2019-04-09 Thread Lukasz Cwik
EAM-7002 > > On Tue, Apr 9, 2019 at 9:50 AM Lukasz Cwik wrote: > >> The SplittableDoFn failures are because of the changes associated with >> BEAM-6978. >> >> On Tue, Apr 9, 2019 at 5:05 AM Michael Luckey

Re: [QUESTION] Should DoFns be able to get the watermark?

2019-04-09 Thread Lukasz Cwik
WatermarkReporterParam is about reporting the watermark. The main usecase is for SplittableDoFns to be able to report the data watermark. The watermark is per input and output of a DoFn. Also each bundle being processed has its local watermarks while the runner computes the global watermark. The r

Re: Updates on Beam Jenkins

2019-04-09 Thread Lukasz Cwik
This is great stuff, my last couple of PRs before this had several > 2hr waits before the tests started up. On Tue, Apr 9, 2019 at 2:39 PM Pablo Estrada wrote: > Thanks for the updates Yifan. I am sure this process has been difficult, > and I appreciate the good communication, and that this didn

Re: [QUESTION] Should DoFns be able to get the watermark?

2019-04-09 Thread Lukasz Cwik
ense... : ) > > If it doesn't make sense, feel free to ignore, and I'll go do my readings. > Thanks! > -P. > > On Tue, Apr 9, 2019 at 1:44 PM Lukasz Cwik wrote: > >> WatermarkReporterParam is about reporting the watermark. The main usecase >> is for Split

Re: [Forked] BEAM-4046 (was [PROPOSAL] Introduce beam-sdks-java gradle project)

2019-04-09 Thread Lukasz Cwik
le/BeamModulePlugin.groovy#L283 > [2] > https://github.com/apache/beam/blob/f9352dc7751c2c35a9189bd405e8a5ef33998b84/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy#L1001 > > On Wed, Apr 3, 2019 at 5:54 PM Lukasz Cwik wrote: > >> As a minor point, we do have some cross language dependen

Re: [QUESTION] Should DoFns be able to get the watermark?

2019-04-09 Thread Lukasz Cwik
teful-processing.html >> https://beam.apache.org/blog/2017/08/28/timely-processing.html >> >> On Tue, Apr 9, 2019 at 3:41 PM Pablo Estrada wrote: >> >>> sunds good. Thanks guys <3 >>> >>> On Tue, Apr 9, 2019 at 3:19 PM Lukasz Cwik wrote: >>&

Re: [ANNOUNCE] New committer announcement: Boyuan Zhang

2019-04-10 Thread Lukasz Cwik
Congratulations and well deserved. On Wed, Apr 10, 2019 at 9:00 AM Kenneth Knowles wrote: > Hi all, > > Please join me and the rest of the Beam PMC in welcoming a new committer: > Boyuan Zhang. > > Boyuan has been contributing to Beam since early 2018. She has proposed > 100+ pull requests acros

Re: [Forked] BEAM-4046 (was [PROPOSAL] Introduce beam-sdks-java gradle project)

2019-04-10 Thread Lukasz Cwik
" on projects only as part of publishing then > everything starts to work? That sounds ideal. > > Kenn > > On Tue, Apr 9, 2019 at 3:49 PM Lukasz Cwik wrote: > >> It would be good if we did as much as possible to make our project as >> much as a conventio

[DISCUSS] Side input consistency guarantees for triggers with multiple firings

2019-04-11 Thread Lukasz Cwik
Today, we define that a side input becomes available to be consumed once at least one firing occurs or when the runner detects that no such output could be produced (e.g. watermark is beyond the end of the window when using the default trigger). For triggers that fire at most once, consumers are gu

Re: [DISCUSS] Side input consistency guarantees for triggers with multiple firings

2019-04-11 Thread Lukasz Cwik
> firing. > > Reuven > > On Thu, Apr 11, 2019 at 9:27 AM Lukasz Cwik wrote: > >> Today, we define that a side input becomes available to be consumed once >> at least one firing occurs or when the runner detects that no such output >> could be produced (e.g. watermark is b

Re: [DISCUSS] Side input consistency guarantees for triggers with multiple firings

2019-04-11 Thread Lukasz Cwik
t; On Thu, Apr 11, 2019 at 9:44 AM Reuven Lax wrote: > >> One thing to keep in mind: triggers that fire multiple times per window >> already tend to be non deterministic. These are element-count or >> processing-time triggers, both of which are fairly non deterministic in &g

Re: [DISCUSS] Side input consistency guarantees for triggers with multiple firings

2019-04-12 Thread Lukasz Cwik
>> If a user is aware of the potential non-deterministic behavior, isn't it >>> almost trivial to refactor his/her user code, by putting PCollectionViews S >>> and T into one single PCollectionView S', to get around the issue? I >>> cannot think of a re

Re: [PROPOSAL] Custom JVM initialization for Beam workers

2019-04-12 Thread Lukasz Cwik
+1 on the use cases that Ahmet pointed out and the solution that Brian put forth. I like how the change is being applied to the Beam Java SDK harness and not just Dataflow so all portable runner users get this as well. On Wed, Apr 10, 2019 at 9:03 PM Kenneth Knowles wrote: > > > On Wed, Apr 10,

Re: [DOC] Portable Spark Runner

2019-04-12 Thread Lukasz Cwik
Thanks for the doc. On Fri, Apr 12, 2019 at 11:34 AM Kyle Weaver wrote: > Hi everyone, > > As some of you know, I've been piggybacking on the existing Spark and > Flink runners to create a portable version of the Spark runner. I wrote up > a summary of the work I've done so far and what remains

Re: [DISCUSS] change the encoding scheme of Python StrUtf8Coder

2019-04-12 Thread Lukasz Cwik
m/apache/beam/blob/069fc3de95bd96f34c363308ad9ba988ab58502d/model/pipeline/src/main/proto/beam_runner_api.proto#L557 >> >> >> >> >> > >> >> >> >> >> > On Thu, 4 Apr 2019 at 13:51, Kenneth Knowles < >> k...@apache.org> wrote: &g

Re: AvroUtils converting generic record to Beam Row causes class cast exception

2019-04-15 Thread Lukasz Cwik
+dev On Sun, Apr 14, 2019 at 10:29 PM Vishwas Bm wrote: > Hi, > > Below is my pipeline: > > KafkaSource (KafkaIO.read) --> Pardo ---> BeamSql > ---> KafkaSink(KafkaIO.write) > > > The avro schema of the topic has a field of logical type > timestamp-millis. KafkaIO.r

Re: DynamoDB Sink Contribution - Contributor Right Request

2019-04-15 Thread Lukasz Cwik
Welcome, I have added you as a contributor to the project and assigned BEAM-7043 to you. On Mon, Apr 15, 2019 at 10:42 AM cm...@godaddy.com wrote: > Hello everyone, > > > > I am an software engineer at Godaddy. Our team is working with and > supporting Beam. I just opened a Jira ticket to build

Re: New contributor to Beam

2019-04-17 Thread Lukasz Cwik
Welcome, I have added you as a contributor to the Apache Beam project. On Wed, Apr 17, 2019 at 12:26 PM Rose Nguyen wrote: > Welcome, Cyrus!! > > On Wed, Apr 17, 2019 at 11:58 AM Niklas Hansson < > niklas.sven.hans...@gmail.com> wrote: > >> Welcome :) >> >> Den ons 17 apr. 2019 kl 20:33 skrev Ai

Re: Python SDK timestamp precision

2019-04-17 Thread Lukasz Cwik
Kenn, this discussion is about the precision of the timestamp in the user data. As you had mentioned, Runners need not have the same granularity of user data as long as they correctly round the timestamp to guarantee that triggers are executed correctly but the user data should have the same precis

Re: Python SDK timestamp precision

2019-04-17 Thread Lukasz Cwik
imestamps unexpectedly. On Wed, Apr 17, 2019 at 1:24 PM Lukasz Cwik wrote: > Kenn, this discussion is about the precision of the timestamp in the user > data. As you had mentioned, Runners need not have the same granularity of > user data as long as they correctly round the timestamp

Re: Go SDK status

2019-04-17 Thread Lukasz Cwik
Thanks for the indepth summary. On Mon, Apr 15, 2019 at 4:19 PM Robert Burke wrote: > Hi Thomas! I'm so glad you asked! > > The status of the Go SDK is complicated, so this email can't be brief. > There's are several dimensions to consider: as a Go Open Source Project, > User Libraries and Exper

Re: SNAPSHOTS have not been updated since february

2019-04-18 Thread Lukasz Cwik
The permissions issue is that the credentials needed to publish to the maven repository are only deployed on machines managed by Apache Infra. Now that the machines have been given back to each project to manage Yifan was investigating some other way to get the permissions on to the machine. On Th

Re: Artifact staging in cross-language pipelines

2019-04-18 Thread Lukasz Cwik
We can expose the artifact staging endpoint and artifact token to allow the expansion service to upload any resources its environment may need. For example, the expansion service for the Beam Java SDK would be able to upload jars. In the "docker" environment, the Apache Beam Java SDK harness conta

Re: Artifact staging in cross-language pipelines

2019-04-18 Thread Lukasz Cwik
represented is SDK > dependent. So if the pipeline SDK tries to do this we have to add n x (n > -1) configurations (for n SDKs). > > - Cham > > On Thu, Apr 18, 2019 at 11:45 AM Lukasz Cwik wrote: > >> We can expose the artifact staging endpoint and artifact token to allow

Re: Artifact staging in cross-language pipelines

2019-04-18 Thread Lukasz Cwik
facts to > artifacts staging service > > > On Thu, Apr 18, 2019 at 2:33 PM Chamikara Jayalath > wrote: > >> >> >> On Thu, Apr 18, 2019 at 2:12 PM Lukasz Cwik wrote: >> >>> Note that Max did ask whether making the expansion service do the >&g

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
Yes, using T makes sense. The WindowedValue was meant to be a context object in the SDK harness that propagates various information about the current element. We have discussed in the past about: * making optimizations which would pass around less of the context information if we know that the DoF

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
Since I don't think this is a contentious change. On Fri, Apr 19, 2019 at 9:25 AM Lukasz Cwik wrote: > Yes, using T makes sense. > > The WindowedValue was meant to be a context object in the SDK harness that > propagates various information about the current element. We have d

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
This seems worthwhile IMO. Ahmet, Pyup[1] is free for open source projects and has an API that allows for dependency checking. They can scan Github repos automatically it seems but it may not be compatible with how Apache permissions with Github work. I'm not sure if there is such a thing for Go.

Re: [DISCUSS] Turn `WindowedValue` into `T` in the FnDataService and BeamFnDataClient interface definition

2019-04-19 Thread Lukasz Cwik
; I would like to create/assign tickets for this work. > > Thanks, > Jincheng > > Lukasz Cwik 于2019年4月20日周六 上午12:26写道: > >> Since I don't think this is a contentious change. >> >> On Fri, Apr 19, 2019 at 9:25 AM Lukasz Cwik wrote: >> >>> Yes, us

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
an > update is possible, if one doesn't exist in the go tooling already. > > On Fri, 19 Apr 2019 at 09:44, Lukasz Cwik wrote: > >> This seems worthwhile IMO. >> >> Ahmet, Pyup[1] is free for open source projects and has an API that >> allows for dependenc

Re: CVE audit gradle plugin

2019-04-19 Thread Lukasz Cwik
7;s not too bad if the > time is put in though. > > On Fri, 19 Apr 2019 at 10:17, Lukasz Cwik wrote: > >> Robert, I believe what is being suggested is a tool that integrates into >> CVE reports automatically and tells us if we have a dependency with a >> security iss

Re: Hello from Hannah Jiang

2019-04-26 Thread Lukasz Cwik
Welcome Hannah. On Fri, Apr 26, 2019 at 5:00 AM Ismaël Mejía wrote: > Welcome to Beam Hannah ! > > On Fri, Apr 26, 2019 at 1:44 PM Maximilian Michels wrote: > > > > Awesome. Welcome Hannah! > > > > Cheers, > > Max > > > > On 26.04.19 05:08, Yifan Zou wrote: > > > Welcome! > > > > > > On Thu, Ap

Re: [DISCUSS] Reconciling ValueState in Java and Python (was: [docs] Python State & Timers)

2019-04-26 Thread Lukasz Cwik
I am in the camp where we should only support merging state (either naturally via things like bags or via combiners). I believe that having the wrapper that Brian suggests is useful for users. As for the @OnMerge method, I believe combiners should have the ability to look at the window information

Re: Artifact staging in cross-language pipelines

2019-04-26 Thread Lukasz Cwik
We should stick with URN + payload + artifact metadata[1] where the only mandatory one that all SDKs and expansion services understand is the "bytes" artifact type. This allows us to add optional URNs for file://, http://, Maven, PyPi, ... in the future. I would make the artifact staging service us

Re: Enable security for data channels in portability

2019-04-26 Thread Lukasz Cwik
The link to the ApiServiceDescriptor is https://github.com/apache/beam/blob/476e17ed6badd4d5c06c4caf8a824805f40a8e7a/model/pipeline/src/main/proto/endpoints.proto#L31 On Fri, Apr 26, 2019 at 4:32 PM Lukasz Cwik wrote: > I had originally taken a look at this a while ago but not much

Re: Enable security for data channels in portability

2019-04-26 Thread Lukasz Cwik
I had originally taken a look at this a while ago but not much has progressed since then. The original idea was that the ApiServiceDescriptor would be extended to support secure ways of authentication/communication. I was prototyping with an OAuth2 client credentials grant at the time but dropped i

Re: Updates on Beam Jenkins

2019-04-29 Thread Lukasz Cwik
Thanks Yifan for driving this. On Mon, Apr 29, 2019 at 8:01 AM Yifan Zou wrote: > Hi all, > > > We now fully switched the Jenkins to new agents. The old agents are > deprecated and VMs will be deleted shortly to make more CPU available in > the us-central1 for tests. Please let me know if you se

Re: [discuss] A tweak to the Python API for SDF?

2019-04-29 Thread Lukasz Cwik
Would it make sense to also do this in the Java SDK? The would make the restriction provider also mirror the TimerSpec and StateSpec which use annotations similar to how its done in Python. On Mon, Apr 29, 2019 at 3:42 AM Robert Bradshaw wrote: > +1 to introducing this Param for consistency (an

Re: Enable security for data channels in portability

2019-04-29 Thread Lukasz Cwik
which can be distributed out of >> band by the infrastructure and can be provided via file system, rpc to >> another service etc. >> gRPC already has some mechanism for standard and custom authentication[1]. >> Instrumenting gRPC channel using command line option or environm

Re: [discuss] A tweak to the Python API for SDF?

2019-04-29 Thread Lukasz Cwik
ve proposed a pull request: > https://github.com/apache/beam/pull/8430 - it was smaller than I thought > : ) - All comments welcome please. > > +Boyuan Zhang I am happy to wait for your > SyntheticSource PR to be merged and make the appropriate changes if you'd > like. > B

Re: Pipeline options validation

2019-04-29 Thread Lukasz Cwik
Kyle your right and it makes sense from the doc but from a user point of view the validation is really asking if the field has been set or not. Differentiation between unset and set has come up in the past for PipelineOptions. On Mon, Apr 29, 2019 at 5:19 PM Kyle Weaver wrote: > Validation.Requi

Re: Pipeline options validation

2019-04-29 Thread Lukasz Cwik
, 2019 at 5:21 PM Lukasz Cwik wrote: > Kyle your right and it makes sense from the doc but from a user point of > view the validation is really asking if the field has been set or not. > Differentiation between unset and set has come up in the past for > PipelineOptions. > > On Mon,

Re: Pipeline options validation

2019-04-30 Thread Lukasz Cwik
rote: > >> Does it make use of the @Nullable annotation or just assume any object >> reference could be null? Now that we are on Java 8 can it use Optional as >> well? (pet issue of mine :-) >> >> On Mon, Apr 29, 2019 at 5:29 PM Lukasz Cwik wrote: >> >>

Re: Pipeline options validation

2019-04-30 Thread Lukasz Cwik
may be a blocker. Or not? > > Regards, > Anton > > On Tue, Apr 30, 2019 at 12:18 PM Lukasz Cwik wrote: > >> The migration to requiring @Nullable on methods that could take/return >> null didn't update PipelineOptions contract and its validation to respect >&g

Re: Artifact staging in cross-language pipelines

2019-04-30 Thread Lukasz Cwik
Agree on adding the 5.5 and the resolution of conflicts/duplicates could be done by either the runner or the artifact staging service. On Tue, Apr 30, 2019 at 10:03 AM Chamikara Jayalath wrote: > > On Fri, Apr 26, 2019 at 4:14 PM Lukasz Cwik wrote: > >> We should stick wit

Re: [Discuss] Publishing pre-release artifacts to repositories

2019-04-30 Thread Lukasz Cwik
The Java SDK uses the ASF managed Nexus repository. There is a snapshot one (where we publish nightly builds) and also a release one (where we place our release candidates). Once the release candidate is approved the Nexus repository has a way to publish it making it an official release. More detai

Re: [DISCUSS] Reconciling ValueState in Java and Python (was: [docs] Python State & Timers)

2019-05-01 Thread Lukasz Cwik
ld be in the API) could be an AnyCombine for now (safe to >> >> overwrite because it's always the same). >> >> >> >> > However I think as folks explore Beam there will always be little >> things that require Metadata and so having access to something w

Re: [DISCUSS] Reconciling ValueState in Java and Python (was: [docs] Python State & Timers)

2019-05-01 Thread Lukasz Cwik
Note, the example didn't support merging windows so I also ignored it. In the case of merging windows, your solution would depend on whether you needed to know from what window the enriched event was from. On Wed, May 1, 2019 at 8:30 AM Lukasz Cwik wrote: > Isn't a value state jus

Re: [DISCUSS] Reconciling ValueState in Java and Python (was: [docs] Python State & Timers)

2019-05-01 Thread Lukasz Cwik
n. >> > > Race conditions are not inherently a problem. They are neither necessary > nor sufficient for correctness. In this case, it is not the classic sense > of race condition anyhow, it is simply a nondeterministic result, which may > often be perfectly fine. &g

Re: Artifact staging in cross-language pipelines

2019-05-02 Thread Lukasz Cwik
On Thu, May 2, 2019 at 7:20 AM Robert Bradshaw wrote: > On Sat, Apr 27, 2019 at 1:14 AM Lukasz Cwik wrote: > > > > We should stick with URN + payload + artifact metadata[1] where the only > mandatory one that all SDKs and expansion services understand is the > "bytes&q

Re: cancel job

2019-05-02 Thread Lukasz Cwik
+u...@beam.apache.org On Thu, May 2, 2019 at 9:51 AM Lukasz Cwik wrote: > ... build pipeline ... > pipeline_result = p.run() > if job_taking_too_long: > pipeline_result.cancel() > > Python: > https://github.com/apache/beam/blob/95d0ac5e5cb59fd0c6a8a4861a38a7087a6c46b5/sd

Re: cancel job

2019-05-02 Thread Lukasz Cwik
... build pipeline ... pipeline_result = p.run() if job_taking_too_long: pipeline_result.cancel() Python: https://github.com/apache/beam/blob/95d0ac5e5cb59fd0c6a8a4861a38a7087a6c46b5/sdks/python/apache_beam/runners/runner.py#L372 Java: https://github.com/apache/beam/blob/ce77db10cdd5f021f383721a

Re: kafka client interoperability

2019-05-02 Thread Lukasz Cwik
+dev On Thu, May 2, 2019 at 10:34 AM Moorhead,Richard < richard.moorhe...@cerner.com> wrote: > In Beam 2.9.0, this check was made: > > > https://github.com/apache/beam/blob/2ba00576e3a708bb961a3c64a2241d9ab32ab5b3/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.jav

Re: [DISCUSS][SQL] Providing support for DISTINCT aggregations

2019-05-02 Thread Lukasz Cwik
Can you also go into more detail why you think 1) is more challenging to implement? On Thu, May 2, 2019 at 11:58 AM Ahmet Altay wrote: > From my limited understanding, would not the stateful combinefn option > require observing the whole input before being able combine and the risk of > blowing

Re: Better naming for runner specific options

2019-05-03 Thread Lukasz Cwik
The concept of a machine type isn't necessarily limited to Dataflow. If it made sense for a runner, they could use AWS/Azure machine types as well. On Fri, May 3, 2019 at 9:32 AM Ahmet Altay wrote: > This idea was discussed in a PR a few months ago, and JIRA was filed as a > follow up [1]. IMO,

Re: Better naming for runner specific options

2019-05-06 Thread Lukasz Cwik
mailto:chamik...@google.com>> wrote: >> > >> > Also, we do have runner specific options classes where truly >> > runner specific options can go. >> > >> > >> https://github.com/apache/beam/blob/master/runners/google-cloud-dat

Re: Contributing to Beam

2019-05-06 Thread Lukasz Cwik
Welcome. On Mon, May 6, 2019 at 2:23 PM Reuven Lax wrote: > Welcome! > > On Mon, May 6, 2019 at 2:15 PM Kenneth Knowles wrote: > >> Welcome! >> >> On Mon, May 6, 2019 at 9:20 AM Ahmet Altay wrote: >> >>> Welcome Shehzaad! >>> >>> *From: *Ismaël Mejía >>> *Date: *Sat, May 4, 2019 at 2:30 PM >>

Re: Coder Evolution

2019-05-08 Thread Lukasz Cwik
There was a thread about coder update in the past here[1]. Also, Reuven sent out a doc[2] about pipeline drain and update which was discussed in this thread[3]. I believe there have been more references to pipeline update in other threads when people tried to change coder encodings in the past as w

Re: Requesting contributor permission for Beam JIRA tickets

2019-05-08 Thread Lukasz Cwik
Welcome, I have assigned BEAM-7240 to you. On Wed, May 8, 2019 at 10:30 AM Ajo Thomas wrote: > Hello, > > I am Ajo Thomas and I was hoping to work on making some improvements to > the beam KinesisIO java SDK. > I have created a ticket for it [ > https://issues.apache.org/jira/browse/BEAM-7240 ]

Re: [DISCUSS] Portability representation of schemas

2019-05-08 Thread Lukasz Cwik
Are you suggesting that schemas become an explicit field on PCollection or that the coder on PCollections has a well known schema coder type that has a payload that has field names, ids, type, ...? I'm much more for the latter since it allows for versioning schema representations over time without

Re: Unexpected behavior of StateSpecs

2019-05-10 Thread Lukasz Cwik
StateSpec should not be used as a key within any maps. We should use the logical name of the StateSpec relative to the DoFn as its id and should only be using that id for comparisons/lookups. On Fri, May 10, 2019 at 1:07 AM Jan Lukavský wrote: > I'm not sure. Generally it affects any runner that

Re: Coder Evolution

2019-05-10 Thread Lukasz Cwik
try > > very hard never to change import Coders, which sometime makes > > development of parts of Beam much more difficult. I think Beam would > > benefit greatly by having a first-class concept of Coder evolution. > > > > BTW for schemas there is a natural way of def

Re: Unexpected behavior of StateSpecs

2019-05-10 Thread Lukasz Cwik
hat it doesn't hash and compare the StateSpec, but only > the StructureId. That looks like best option to me. But I'm not sure about > other implications this might have. > > Jan > On 5/10/19 5:43 PM, Reuven Lax wrote: > > Ok so this sounds like a bug in the DirectRunner t

Re: Problem with gzip

2019-05-10 Thread Lukasz Cwik
+u...@beam.apache.org Reshuffle on Google Cloud Dataflow for a bounded pipeline waits till all the data has been read before the next transforms can run. After the reshuffle, the data should have been processed in parallel across the workers. Did you see this? Are you able to change the input of

Re: Beam's Conda package

2019-05-10 Thread Lukasz Cwik
I'm not aware of who set up conda as well. There seem to have been ~4500 downloads of the package so that is a good amount of users. On Fri, May 10, 2019 at 11:45 AM Ahmet Altay wrote: > Hi all, > > There a conda package for apache-beam [1]. As far as I know, we do not > release this package. Do

Re: Problem with gzip

2019-05-10 Thread Lukasz Cwik
to run, according to > one test (24 gzip files, 17 million lines in total) I did. > > The file format for our users are mostly gzip format, since uncompressed > files would be too costly to store (It could be in hundreds of GB). > > Thanks, > > Allie > > > *From: *Lukasz Cw

Re: Problem with gzip

2019-05-10 Thread Lukasz Cwik
2019 at 1:25 PM Allie Chen wrote: > Yes, that is correct. > > *From: *Allie Chen > *Date: *Fri, May 10, 2019 at 4:21 PM > *To: * > *Cc: * > > Yes. >> >> *From: *Lukasz Cwik >> *Date: *Fri, May 10, 2019 at 4:19 PM >> *To: *dev >> *Cc: * &

Re: Problem with gzip

2019-05-10 Thread Lukasz Cwik
t;> >>> The file format for our users are mostly gzip format, since uncompressed >>> files would be too costly to store (It could be in hundreds of GB). >>> >>> Thanks, >>> >>> Allie >>> >>> >>> *From: *Lukasz Cwik

Re: Do we maintain offline artifact version in javadocs sdks/java/javadoc/build.gradle

2019-05-13 Thread Lukasz Cwik
What is the difference between the two files you are referring to? Note that sdks/java/javadoc/build.gradle is meant to produce one giant javadoc across many modules that users would be interested in (core/extensions/io/...) meant to be published on the website. *From: *Ankur Goenka *Date: *Fri,

Re: Do we maintain offline artifact version in javadocs sdks/java/javadoc/build.gradle

2019-05-13 Thread Lukasz Cwik
. > > Kenn > > *From: *Lukasz Cwik > *Date: *Mon, May 13, 2019 at 9:04 AM > *To: *dev > > What is the difference between the two files you are referring to? >> >> Note that sdks/java/javadoc/build.gradle is meant to produce one giant >> javadoc across many mo

Re: Coder Evolution

2019-05-13 Thread Lukasz Cwik
of Beam much more difficult. I think Beam would >> > benefit greatly by having a first-class concept of Coder evolution. >> > >> > BTW for schemas there is a natural way of defining evolution that can >> be >> > handled by SchemaCoder. >> > >> &

Re: Do we maintain offline artifact version in javadocs sdks/java/javadoc/build.gradle

2019-05-13 Thread Lukasz Cwik
is https://github.com/lukecwik/incubator-beam/commit/8cc7c53139d0eecad0ec994b9a313cf31645 >From a Javadoc correctness and maintenance point of view, this seems much simpler overall to me. *From: *Lukasz Cwik *Date: *Mon, May 13, 2019 at 1:39 PM *To: *dev I see. We should be able to

Re: Problem with gzip

2019-05-14 Thread Lukasz Cwik
yKey won't wait until all data has been read? > > Thanks! > Allie > > *From: *Lukasz Cwik > *Date: *Fri, May 10, 2019 at 5:36 PM > *To: *dev > *Cc: *user > > There is no such flag to turn of fusion. >> >> Writing 100s of GiBs of uncompressed data to

Re: Jenkins commenting on PRs again

2019-05-14 Thread Lukasz Cwik
I have seen this in the past, I don't remember how it was resolved. Kenn is specifically asking about seeing messages: asfgit commented 5 minutes ago SUCCESS --none-- *From: *Kenneth Knowles *Date: *T

  1   2   3   4   5   6   7   8   9   10   >