Re: Why not adding all coders into ModelCoderRegistrar?

2018-10-15 Thread Lukasz Cwik
You will want to add your own CoderTranslatorRegistrar for the additional
URNs that you want to support to the runner and make sure the SDK that your
using submits those coders using those URNs and payloads. If your using
Java, you would want to make sure that the same CoderTranslatorRegistrar is
on the classpath and discoverable via a ServiceLoader. If your using
Python/Go SDKs for creating pipelines, you'll need to use their coder
registration/translation mechanism (I'm not sure if they support dynamic
registration like the Java SDK does).

On Wed, Oct 3, 2018 at 1:53 PM Shen Li  wrote:

> Hi Lukasz,
>
> Is there a way to get the SDK coders (LengthPrefixCoder,
> LengthPrefixCoder etc.) instead of a
> LengthPrefixCoder on the runner side from
> RunnerApi.Pipeline? Our runner needs to serialize the key and use its hash
> value to keep some per-key states. Now I am getting the ClassCastException
> as the key seen by the runner (an Integer) is not a Byte array.
>
> Thanks,
> Shen
>
> On Fri, Sep 28, 2018 at 2:20 PM Shen Li  wrote:
>
>> Thank you, Lukasz!
>>
>> Best,
>> Shen
>>
>> On Fri, Sep 28, 2018 at 2:11 PM Lukasz Cwik  wrote:
>>
>>> Runners can never know about every coder that a user may want to write
>>> which is why we need to have a mechanism for Runners to be able to convert
>>> any unknown coder to one it can handle. This is done via
>>> WireCoders.instantiateRunnerWireCoder but this modifies the original coder
>>> which is why WireCoders.addSdkWireCoder creates the proto definition that
>>> the SDK should be told to use. In your case, your correct in that KV>> T> becomes KVCoder,
>>> LengthPrefixCoder> on the runner side and on the SDK side
>>> it should be KVCoder,
>>> LengthPrefixCoder>. More details in [1].
>>>
>>> 1:
>>> http://doc/1IGduUqmhWDi_69l9nG8kw73HZ5WI5wOps9Tshl5wpQA#heading=h.sh4d5klmtfis
>>>
>>>
>>>
>>> On Fri, Sep 28, 2018 at 11:02 AM Shen Li  wrote:
>>>
 Hi,

 I noticed that ModelCoderRegistrar only includes 9 out of ~40 coders.
 May I know the rationale behind this decision?


 https://github.com/apache/beam/blob/release-2.7.0/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ModelCoderRegistrar.java

 I think one consequence of the above configuration is
 that WireCoders.instantiateRunnerWireCoder cannot instantiate KV coders
 correctly, where VoidCoder (key coder) becomes
 LengthPrefixCoder(ByteArrayCoder). What is the appropriate way to get
 KvCoder from RunnerApi.Pipeline?

 Thanks,
 Shen

>>>


Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Lukasz Cwik
I see, is the issue that we currently are using a JSON representation for
options when being serialized and when we get some unknown option, we don't
know how to convert it into its JSON form?

On Mon, Oct 15, 2018 at 2:41 PM Robert Bradshaw  wrote:

> On Mon, Oct 15, 2018 at 11:30 PM Lukasz Cwik  wrote:
> >
> > On Mon, Oct 15, 2018 at 1:17 PM Robert Bradshaw 
> wrote:
> >>
> >> On Mon, Oct 15, 2018 at 7:50 PM Lukasz Cwik  wrote:
> >> >
> >> > I agree with the sentiment for better error checking.
> >> >
> >> > We can try to make it such that the SDK can "fetch" the set of
> options that the runner supports by making a call to the Job API. The API
> could return a list of option names (descriptions for --help purposes and
> also potentially the expected format) which would remove the worry around
> "unknown" options. Yes I understand to be able to make the Job API call, we
> may need to parse some options from the args parameters first and then
> parse the unknown options after they are fetched.
> >>
> >> This is an interesting idea, but seems it could get quite complicated.
> >> E.g. for delegating runners, one would first read the options to
> >> determine which runner to fetch the options from, which would then
> >> return a set of options that possibly depends on the values of some of
> >> its options...
> >>
> >> > Alternatively, we can choose an explicit format upfront.
> >> > To expand on the exact format for --runner_option=..., here are some
> different ideas:
> >> > 1) Specified multiple times, each one is an explicit flag
> >> > --runner_option=--blah=bar --runner_option=--foo=baz1
> --runner_option=--foo=baz2
> >>
> >> I'm -1 on this format. We should move away from the idea that options
> >> == flags (as that doesn't compose well with other libraries that do
> >> their own flags parsing). The ability to parse a set of flags into
> >> options is just a convenience that an author may (or may not) choose
> >> to use (e.g. when running pipelines a long-lived process like a
> >> service or a notebook, the command line flags are almost certainly not
> >> the right interface).
> >>
> >> > 2) specified multiple times, we drop the explicit flag
> >> > --runner_option=blah=bar --runner_option=foo=baz1
> --runner_option=foo=baz2
> >>
> >> This or (4) is my preference.
> >>
> >> > 3) we use a string which the runner can choose to interpret however
> they want (JSON/XML shown below)
> >> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
> >> >
> --runner_option='barbaz1baz2'
> >>
> >> This would make validation hard. Also, I think it makes sense for some
> >> runner options to be "shared" (parallelism") by convention, so letting
> >> it be a free-form string wouldn't allow different runners to inspect
> >> different bits.
> >>
> >> We should consider if we should use urns for namespacing, and
> >> assigning semantic meaning to strings, here.
> >>
> >> > 4) we use a string which must be a specific format such as JSON
> (allows the SDK to do simple validation):
> >> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
> >>
> >> I like this in that at least some validation can be performed, and
> >> expectations of how to format richer types. On the other hand it gets
> >> a bit verbose, given that most (I'd imagine) options will be simple.
> >> As with normal options,
> >>
> >> --option1=value1 --option2=value2
> >>
> >> is shorthand for {"option1": value1, "option2": value2}.
> >>
> > I lean to 4 the most. With 2, you run into issues of what does
> --runner_option=foo=["a", "b"] --runner_option=foo=["c", "d"] mean?
> > Is it an error or list of lists or concatenated. Similar issues for map
> types represented via JSON object {...}
>
> We can err to be on the safe side unless/until an argument can be made
> that merging is more natural. I just think this will be excessively
> verbose to use.
>
> >> > I would strongly suggest that we go with the "fetch" approach, since
> this makes the set of options discoverable and helps users find errors much
> earlier in their pipeline.
> >>
> >> This seems like an advanced feature that SDKs may want to support, but
> >> I wouldn't want to require this complexity for bootstrapping an SDK.
> >>
> > SDKs that are starting off wouldn't need to "fetch" options, they could
> choose to not support runner options or they could choose to pass all
> options through to the runner blindly. Fetching the options only provides
> the SDK the ability to provide error checking upfront and useful error/help
> messages.
>
> But how to even pass all options through blindly is exactly the
> difficulty we're running into here.
>
> >> Regarding always keeping runner options separate, +1, though I'm not
> >> sure the line is always clear.
> >>
> >>
> >> > On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw 
> wrote:
> >> >>
> >> >> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels 
> wrote:
> >> >> >
> >> >> > I agree that the current approach breaks the pipeline options
> 

Re: [Proposal] Euphoria DSL - looking for reviewers

2018-10-15 Thread Reuven Lax
There isn't currently a concept of a partial committer - you are either a
committer or you aren't. Figuring out the best way forward here is a
question for the wider community.

Reuven

On Sun, Oct 14, 2018 at 2:41 AM David Morávek 
wrote:

> Thanks Kenn and Reuven!
>
> This brings up the question, how should we proceed with the further
> development? Up until now, we did all changes in our own repository, which
> was very flexible as we could do code reviews and PR merges by ourselves.
>
> We would love to take a full responsibility for the newly created modules,
> because we have put a great effort into their development over the years.
>
> Would it be possible to gain commit rights for these modules, so we could
> maintain them without having to bother a committer with each patch or
> improvement?
>
> D.
>
> On Sun, Oct 14, 2018 at 10:48 AM Reuven Lax  wrote:
>
>> This is a brand new extension, so I don't think it's necessary for a Beam
>> committer to review every line of this before merging. A committer should
>> ensure that files are in the correct places, IP clearance is done, etc.,
>> and then I  think it's fine to merge.
>>
>> I do think this code needs to be reviewed in detail, but I think it's
>> sufficient to trust the Euphoria authors to do this review themselves. If
>> the code has already been peer reviewed between the Euphoria authors, I
>> feel like Beam's review before submit policy has been satisfied.
>>
>> Reuven
>>
>> On Wed, Oct 10, 2018 at 1:26 AM Plajt, Vaclav <
>> vaclav.pl...@firma.seznam.cz> wrote:
>>
>>> Hello Beam devs,
>>> we finished our main goals in development of Euphoria DSL. It is Easy to
>>> use Java 8 API build on top of the Beam's Java SDK. API provides a
>>> high-level abstraction of data transformations, with focus on the Java 8
>>> language features (e.g. lambdas and streams). It is fully inter-operable
>>> with existing Beam SDK and convertible back and forth. It allows fast
>>> prototyping through use of (optional) Kryo based coders and can be
>>> seamlessly integrated into existing Beam Pipelines.
>>>
>>> Now we believe that it is the time to start discussion about it with the
>>> community. Which will hopefully lead to vote about adapting it into Apache
>>> Beam project. Most of main ideas and development goals were presented in
>>> Beam Summit in London [1].
>>>
>>> We are looking for reviewers within the community. Please start with
>>> documentation [2] or design document [3]. Our contribution is divided to
>>> two modules: `org.apache.beam:beam-sdks-java-extensions-euphoria` and
>>> `org.apache.beam:beam-sdks-java-extensions-kryo`. Rest of the code base
>>> remains untouched.
>>> All the checks in MR [5] are passing with exception of "Website
>>> PreCommit". Which seems to be broken, little help here would be appreciated.
>>>
>>> Thank you
>>> We are looking forward for your feedback.
>>> {david.moravek,vaclav.plajt,marek.simunek}@firma.seznam.cz
>>>
>>> Resources:
>>> [1] Beam Summit London presentation:
>>> https://docs.google.com/presentation/d/1SagpmzJ-tUQki5VsQOEEEUyi_LXRJdG_3OBLdjBKoh4/edit?usp=sharing
>>> [2] Documentation:
>>> https://github.com/seznam/beam/blob/dsl-euphoria/website/src/documentation/sdks/euphoria.md
>>> [3] Design Document: https://s.apache.org/beam-euphoria
>>> [4] ASF Jira Issue: https://issues.apache.org/jira/browse/BEAM-3900
>>> [5] Pull Request: https://github.com/apache/beam/pull/6601
>>> [6] Original proposal:
>>> http://mail-archives.apache.org/mod_mbox/beam-dev/201712.mbox/%3ccajjqkhnrp1z8atteogmpfkqxrcjeanb3ykowvvtnwyrvv_-...@mail.gmail.com%3e
>>>
>>>
>>>
>>> Je dobré vědět, že tento e-mail a přílohy jsou důvěrné. Pokud spolu
>>> jednáme o uzavření obchodu, vyhrazujeme si právo naše jednání kdykoli
>>> ukončit. Pro fanoušky právní mluvy - vylučujeme tím ustanovení občanského
>>> zákoníku o předsmluvní odpovědnosti. Pravidla o tom, kdo u nás a jak
>>> vystupuje za společnost a kdo může co a jak podepsat naleznete zde
>>> 
>>>
>>> You should know that this e-mail and its attachments are confidential.
>>> If we are negotiating on the conclusion of a transaction, we reserve the
>>> right to terminate the negotiations at any time. For fans of legalese—we
>>> hereby exclude the provisions of the Civil Code on pre-contractual
>>> liability. The rules about who and how may act for the company and what are
>>> the signing procedures can be found here
>>> .
>>>
>>


Re: Can we allow SimpleFunction and SerializableFunction to throw Exception?

2018-10-15 Thread Reuven Lax
Compilation compatibility is a big part of what Beam aims to provide with
its guarantees. Romain makes a good point that most users are not invoking
SeralizableFunctions themselves (they are usually invoked inside of Beam
classes such as MapElements), however I suspect some users do these things.

On Sun, Oct 14, 2018 at 2:38 PM Kenneth Knowles  wrote:

> Romain has brought up two good aspects of backwards compatibility:
>
>  - runtime replacement vs recompile
>  - consumer (covariant) vs producer (contravariant, such as interfaces a
> user implements)
>
> In this case, I think the best shoice is covariant and contravariant
> (invariant) backwards compat including recompile compat. But we shouldn't
> assume there is one obvious definition of "backwards compatibility".
>
> Does it help to introduce a new functional interface?
>
> Kenn
>
> On Sun, Oct 14, 2018 at 6:25 AM Romain Manni-Bucau 
> wrote:
>
>> Beam does not catch Exception for function usage so it will have to do it
>> in some places.
>>
>> A user does not have to execute the function so worse case it impacts
>> tests and in any case the most important: it does not impact the user until
>> it recompiles the code (= runtime is not impacted).
>>
>> Romain Manni-Bucau
>> @rmannibucau  |  Blog
>>  | Old Blog
>>  | Github
>>  | LinkedIn
>>  | Book
>> 
>>
>>
>> Le dim. 14 oct. 2018 à 15:19, Reuven Lax  a écrit :
>>
>>> What in Beam codebase is not ready, and how do we know that user code
>>> doesn't have the same issue?
>>>
>>> On Sun, Oct 14, 2018 at 6:04 AM Romain Manni-Bucau <
>>> rmannibu...@gmail.com> wrote:
>>>
 Hmm, tested also and it works until something in the codeflow does not
 respect that constraint - see
 com.sun.tools.javac.comp.Flow.FlowAnalyzer#errorUncaught. In other words
 beam codebase is not ready for that and will make it fail but it is ok
 cause we can fix it but user code does not rely on that so it is fine to
 update it normally.

 Romain Manni-Bucau
 @rmannibucau  |  Blog
  | Old Blog
  | Github
  | LinkedIn
  | Book
 


 Le dim. 14 oct. 2018 à 14:39, Reuven Lax  a écrit :

> Just tried it, doesn't appear to work :(
>
> error: unreported exception Exception; must be caught or declared to
> be thrown
>
> On Sun, Oct 14, 2018 at 1:55 AM Romain Manni-Bucau <
> rmannibu...@gmail.com> wrote:
>
>> not with java>=8 AFAIK
>>
>> Romain Manni-Bucau
>> @rmannibucau  |  Blog
>>  | Old Blog
>>  | Github
>>  | LinkedIn
>>  | Book
>> 
>>
>>
>> Le dim. 14 oct. 2018 à 10:49, Reuven Lax  a écrit :
>>
>>> But it means that other functions that call SerializableFunctions
>>> must now declare exceptions, right? If yes, this is incompatible.
>>>
>>> On Sun, Oct 14, 2018 at 1:37 AM Romain Manni-Bucau <
>>> rmannibu...@gmail.com> wrote:
>>>
 No, only parameter types and return type is used to lookup methods.

 Romain Manni-Bucau
 @rmannibucau  |  Blog
  | Old Blog
  | Github
  | LinkedIn
  | Book
 


 Le dim. 14 oct. 2018 à 09:45, Reuven Lax  a
 écrit :

> I've run into this problem before as well. Doesn't changing the
> signature involve a backwards-incompatible change though?
>
> On Wed, Oct 3, 2018 at 5:11 PM Jeff Klukas 
> wrote:
>
>> I'm working on https://issues.apache.org/jira/browse/BEAM-5638
>> to add exception handling options to single message transforms in 
>> the Java
>> SDK.
>>
>> MapElements' via() method is overloaded to accept either a
>> SimpleFunction, a SerializableFunction, or a Contextful, all of 
>> which are
>> ultimately stored as a Contextful where the mapping functionis 
>> expected to
>> have signature:
>>

[PROPOSAL] Using Bazel and Docker for Python SDK development and tests

2018-10-15 Thread Udi Meiri
Hi,

In light of increasing Python pre-commit times due to the added Python 3
tests,
I thought it might be time to re-evaluate the tools used for Python tests
and development, and propose an alternative.

Currently, we use nosetests, tox, and virtualenv for testing.
The proposal is to use Bazel, which I believe can replace the above tools
while adding:
- parallel testing: each target has its own build directory, providing
isolation from build artifacts such as from Cython
- incremental testing: it is possible to precisely define each test's
dependencies

There's also a requirement to test against specific Python versions, such
as 2.7 and 3.4.
This could be done using docker containers having the precise version of
interpreter and Bazel.

In summary:
Bazel could replace the need for virtualenv, tox, and nosetests.
The addition of Docker images would allow testing against specific Python
versions.


smime.p7s
Description: S/MIME Cryptographic Signature


Re: Vendoring vulnerable Guava (CVE-2018-10237)

2018-10-15 Thread Lukasz Cwik
Nice on the gPRC update to much newer Guava. Once that is out, would be
worthwhile to bump up our usage as well.

On Mon, Oct 15, 2018 at 2:44 PM Andrew Pilloud  wrote:

> gRPC 1.15 was stuck at 20.0 for Java 6 support, but supports 24.1.1+
> .
> grpc 1.16 will be out in about a week with a dependency on Guava 26.0 (
> https://github.com/grpc/grpc-java/blob/v1.16.x/build.gradle#L114).
>
> I stuck the change into a PR to see what would break, looks like a lot of
> things are unhappy: https://github.com/apache/beam/pull/6695
>
> Andrew
>
> On Mon, Oct 15, 2018 at 2:11 PM Lukasz Cwik  wrote:
>
>> For example, we vendor gRPC and it still depends on 20.0 in its latest
>> version (https://mvnrepository.com/artifact/io.grpc/grpc-core/1.15.1).
>>
>> On Mon, Oct 15, 2018 at 2:10 PM Lukasz Cwik  wrote:
>>
>>> 20.0 is a common version used by many of our dependencies, using 20.0 is
>>> least likely to cause classpath issues. Note that with Guava 22.0+, they
>>> have said they won't introduce backwards incompatible changes anymore so
>>> getting past 22.0 would mean we could just rely on using the latest at all
>>> times.
>>>
>>> I'm not sure the cost of upgrading our dependencies to be compatible
>>> with 22.0+ though.
>>>
>>> On Mon, Oct 15, 2018 at 11:11 AM Andrew Pilloud 
>>> wrote:
>>>
 We vendor a known vulnerable version of Guava. The specific
 vulnerability is low to no impact on Beam but it does potentially affect
 any server that uses Java serialization with Beam on the classpath. Do we
 have a reason for still being on Guava 20.0?

 https://github.com/google/guava/wiki/CVE-2018-10237

 Andrew

>>>


Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Chamikara Jayalath
+1 for new lists.

Thanks,
Cham

On Mon, Oct 15, 2018 at 12:09 PM Ismaël Mejía  wrote:

> +1
> On Mon, Oct 15, 2018 at 8:14 PM Mikhail Gryzykhin 
> wrote:
> >
> > +1 also suggested that in another thread.
> >
> > --Mikhail
> >
> > Have feedback?
> >
> >
> > On Mon, Oct 15, 2018 at 11:10 AM Rui Wang  wrote:
> >>
> >> +1
> >>
> >> Agree that people might be only interested in JIRA activities but not
> in the build.
> >>
> >>
> >> -Rui
> >>
> >> On Mon, Oct 15, 2018 at 10:27 AM Andrew Pilloud 
> wrote:
> >>>
> >>> +1. A bunch of Jenkins spam goes to dev as well. Would be good to it
> all to a new list.
> >>>
> >>> Andrew
> >>>
> >>> On Mon, Oct 15, 2018 at 10:00 AM Colm O hEigeartaigh <
> cohei...@apache.org> wrote:
> 
>  As a rationale, some users might be interested in seeing JIRA
> activity but might not care about whether the build is broken or not :-)
> 
>  Colm.
> 
>  On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles 
> wrote:
> >
> > Separating this out from the Jira notification thread.
> >
> > Colm suggests that we also separate build notifications.
> >
> > WDYT?
> >
> > Kenn
> 
> 
> 
>  --
>  Colm O hEigeartaigh
> 
>  Talend Community Coder
>  http://coders.talend.com
>


Re: Rethinking Timers as PCollections

2018-10-15 Thread Lukasz Cwik
As Kenn mentioned, the timer is tracking the watermark of the main input
PCollection and not the input watermark of the ParDo which would allow it
to fire and thus it couldn't block itself.

I didn't find the wiring in the Java SDK to be difficult since Flatten
already required supporting multiple input producers for operations.

On Mon, Oct 8, 2018 at 3:41 PM Kenneth Knowles  wrote:

> Really love this thread. The analysis is really educational. Seems like
> the pun of "PCollection" for so many purposes is hitting its limit.
>
> Timers should fire according to just the watermark of the data input, but
> nevertheless are a hold on GC and also output watermark.
>
> Kenn
>
> On Thu, Oct 4, 2018 at 3:12 AM Robert Bradshaw 
> wrote:
>
>> Yes, this is all about how timers are represented in the model, as
>> reified in the proto(s).
>>
>> Coming back around to this, I've started looking at what an alternative
>> implementation/representations could look like.
>>
>> If we were to model timers as PCollections in the runner graph, it seems
>> a self loop is too simple. Instead, one would have an output timer
>> PCollection that gets fed into a special "per window-key timer
>> consolidating" operation that then produces the input timer PCollection.
>> Also, the input watermark being the min of the inputs doesn't quite work
>> out, as a timer whose timestamp is less than its firing time should not
>> prevent itself from firing (by holding up the input watermark). It seems
>> there's a lot of caveats and complexity to try to fit it in the model this
>> way (both conceptually, and implementation wise (e.g. one can no longer
>> serialize a ParDo operation without augmenting the larger graph)), as
>> opposed to letting timers be an "internal" property of some DoFns similar
>> to how state is currently modeled.
>>
>> On the flip side, at execution time, I see the value in having
>> PCollections (or at least explicit Input/Output ports) to be able to attach
>> the choice of (windowed) timer encoding. This comes at the cost supporting
>> multiple distinct input producers for operations (possibly unnecessarily
>> complexity if timers are the only use of this) and the wiring of the timer
>> input/output ports to the DoOperation was a bit awkward (in the Python SDK
>> at least).
>>
>>
>>
>> On Fri, Sep 21, 2018 at 6:06 PM Maximilian Michels 
>> wrote:
>>
>>> Very interesting thread.
>>>
>>> Having read the original Timer design document, I find it compelling to
>>> model timers with a loop from producing to consuming PCollections. This
>>> makes it very explicit how timers are positioned in the dataflow.
>>>
>>> What Robert proposes looks less explicit, yet much closer to how Runner
>>> authors would go about to implement it. I'm not fully aware of any
>>> limitations of this model. Lukasz mentioned that we would have to hold
>>> back the Watermark for as long as the Timer is not yet set, as
>>> potentially it could have already been passed before set. As for output
>>> time being different from fire time, I suppose we can add a hold for the
>>> output watermark before the timer is fired.
>>>
>>> Whichever model we pursue, we have to solve the same
>>> problems/requirements for Timers. It does look like this is more a
>>> problem of how things are represented in the proto? Practically, the
>>> runtime implementation looks similar.
>>>
>>> If I had to choose I'd probably go for timers being represented as part
>>> of a spec for a DoFn (which seems to be already the case). Timers as
>>> separate PCollections seems elegant but less practical to me.
>>>
>>> -Max
>>>
>>> [Disclaimer: I could be wrong since I just thought about this in more
>>> detail]
>>>
>>> On 20.09.18 00:28, Robert Bradshaw wrote:
>>> > On Wed, Sep 19, 2018 at 11:54 PM Lukasz Cwik >> > > wrote:
>>> >
>>> >
>>> > On Wed, Sep 19, 2018 at 2:46 PM Robert Bradshaw <
>>> rober...@google.com
>>> > > wrote:
>>> >
>>> > On Wed, Sep 19, 2018 at 8:31 PM Lukasz Cwik >> > > wrote:
>>> >
>>> > *How does modelling a timer as a PCollection help the Beam
>>> > model?*
>>> >
>>> > The largest concern was about how to model timers within
>>> > Apache Beam that:
>>> > 1) removed the need for the watermark hold that is
>>> typically
>>> > accompanied with state/timer implementations
>>> > 2) enabled the ability to set the explicit output time to
>>> be
>>> > independent of the firing time for all timer
>>> specifications [1]
>>> >
>>> > I felt as though treating timers as a self-loop around the
>>> > ParDo PTransform allowed us to use the natural definition
>>> of
>>> > output watermark = min(all input watermarks) as a way to
>>> > define how timers hold output and using windowed values
>>> that
>>> > contained timers as a 

Re: Vendoring vulnerable Guava (CVE-2018-10237)

2018-10-15 Thread Andrew Pilloud
gRPC 1.15 was stuck at 20.0 for Java 6 support, but supports 24.1.1+
.
grpc 1.16 will be out in about a week with a dependency on Guava 26.0 (
https://github.com/grpc/grpc-java/blob/v1.16.x/build.gradle#L114).

I stuck the change into a PR to see what would break, looks like a lot of
things are unhappy: https://github.com/apache/beam/pull/6695

Andrew

On Mon, Oct 15, 2018 at 2:11 PM Lukasz Cwik  wrote:

> For example, we vendor gRPC and it still depends on 20.0 in its latest
> version (https://mvnrepository.com/artifact/io.grpc/grpc-core/1.15.1).
>
> On Mon, Oct 15, 2018 at 2:10 PM Lukasz Cwik  wrote:
>
>> 20.0 is a common version used by many of our dependencies, using 20.0 is
>> least likely to cause classpath issues. Note that with Guava 22.0+, they
>> have said they won't introduce backwards incompatible changes anymore so
>> getting past 22.0 would mean we could just rely on using the latest at all
>> times.
>>
>> I'm not sure the cost of upgrading our dependencies to be compatible with
>> 22.0+ though.
>>
>> On Mon, Oct 15, 2018 at 11:11 AM Andrew Pilloud 
>> wrote:
>>
>>> We vendor a known vulnerable version of Guava. The specific
>>> vulnerability is low to no impact on Beam but it does potentially affect
>>> any server that uses Java serialization with Beam on the classpath. Do we
>>> have a reason for still being on Guava 20.0?
>>>
>>> https://github.com/google/guava/wiki/CVE-2018-10237
>>>
>>> Andrew
>>>
>>


Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Robert Bradshaw
On Mon, Oct 15, 2018 at 11:30 PM Lukasz Cwik  wrote:
>
> On Mon, Oct 15, 2018 at 1:17 PM Robert Bradshaw  wrote:
>>
>> On Mon, Oct 15, 2018 at 7:50 PM Lukasz Cwik  wrote:
>> >
>> > I agree with the sentiment for better error checking.
>> >
>> > We can try to make it such that the SDK can "fetch" the set of options 
>> > that the runner supports by making a call to the Job API. The API could 
>> > return a list of option names (descriptions for --help purposes and also 
>> > potentially the expected format) which would remove the worry around 
>> > "unknown" options. Yes I understand to be able to make the Job API call, 
>> > we may need to parse some options from the args parameters first and then 
>> > parse the unknown options after they are fetched.
>>
>> This is an interesting idea, but seems it could get quite complicated.
>> E.g. for delegating runners, one would first read the options to
>> determine which runner to fetch the options from, which would then
>> return a set of options that possibly depends on the values of some of
>> its options...
>>
>> > Alternatively, we can choose an explicit format upfront.
>> > To expand on the exact format for --runner_option=..., here are some 
>> > different ideas:
>> > 1) Specified multiple times, each one is an explicit flag
>> > --runner_option=--blah=bar --runner_option=--foo=baz1 
>> > --runner_option=--foo=baz2
>>
>> I'm -1 on this format. We should move away from the idea that options
>> == flags (as that doesn't compose well with other libraries that do
>> their own flags parsing). The ability to parse a set of flags into
>> options is just a convenience that an author may (or may not) choose
>> to use (e.g. when running pipelines a long-lived process like a
>> service or a notebook, the command line flags are almost certainly not
>> the right interface).
>>
>> > 2) specified multiple times, we drop the explicit flag
>> > --runner_option=blah=bar --runner_option=foo=baz1 --runner_option=foo=baz2
>>
>> This or (4) is my preference.
>>
>> > 3) we use a string which the runner can choose to interpret however they 
>> > want (JSON/XML shown below)
>> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>> > --runner_option='barbaz1baz2'
>>
>> This would make validation hard. Also, I think it makes sense for some
>> runner options to be "shared" (parallelism") by convention, so letting
>> it be a free-form string wouldn't allow different runners to inspect
>> different bits.
>>
>> We should consider if we should use urns for namespacing, and
>> assigning semantic meaning to strings, here.
>>
>> > 4) we use a string which must be a specific format such as JSON (allows 
>> > the SDK to do simple validation):
>> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>>
>> I like this in that at least some validation can be performed, and
>> expectations of how to format richer types. On the other hand it gets
>> a bit verbose, given that most (I'd imagine) options will be simple.
>> As with normal options,
>>
>> --option1=value1 --option2=value2
>>
>> is shorthand for {"option1": value1, "option2": value2}.
>>
> I lean to 4 the most. With 2, you run into issues of what does 
> --runner_option=foo=["a", "b"] --runner_option=foo=["c", "d"] mean?
> Is it an error or list of lists or concatenated. Similar issues for map types 
> represented via JSON object {...}

We can err to be on the safe side unless/until an argument can be made
that merging is more natural. I just think this will be excessively
verbose to use.

>> > I would strongly suggest that we go with the "fetch" approach, since this 
>> > makes the set of options discoverable and helps users find errors much 
>> > earlier in their pipeline.
>>
>> This seems like an advanced feature that SDKs may want to support, but
>> I wouldn't want to require this complexity for bootstrapping an SDK.
>>
> SDKs that are starting off wouldn't need to "fetch" options, they could 
> choose to not support runner options or they could choose to pass all options 
> through to the runner blindly. Fetching the options only provides the SDK the 
> ability to provide error checking upfront and useful error/help messages.

But how to even pass all options through blindly is exactly the
difficulty we're running into here.

>> Regarding always keeping runner options separate, +1, though I'm not
>> sure the line is always clear.
>>
>>
>> > On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw  
>> > wrote:
>> >>
>> >> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels  
>> >> wrote:
>> >> >
>> >> > I agree that the current approach breaks the pipeline options contract
>> >> > because "unknown" options get parsed in the same way as options which
>> >> > have been defined by the user.
>> >>
>> >> FWIW, I think we're already breaking this "contract." Unknown options
>> >> are silently ignored; with this change we just change how we record
>> >> them. It still feels a bit hacky though.
>> >>
>> >> > I'm not sure the 

Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Lukasz Cwik
On Mon, Oct 15, 2018 at 1:17 PM Robert Bradshaw  wrote:

> On Mon, Oct 15, 2018 at 7:50 PM Lukasz Cwik  wrote:
> >
> > I agree with the sentiment for better error checking.
> >
> > We can try to make it such that the SDK can "fetch" the set of options
> that the runner supports by making a call to the Job API. The API could
> return a list of option names (descriptions for --help purposes and also
> potentially the expected format) which would remove the worry around
> "unknown" options. Yes I understand to be able to make the Job API call, we
> may need to parse some options from the args parameters first and then
> parse the unknown options after they are fetched.
>
> This is an interesting idea, but seems it could get quite complicated.
> E.g. for delegating runners, one would first read the options to
> determine which runner to fetch the options from, which would then
> return a set of options that possibly depends on the values of some of
> its options...
>
> > Alternatively, we can choose an explicit format upfront.
> > To expand on the exact format for --runner_option=..., here are some
> different ideas:
> > 1) Specified multiple times, each one is an explicit flag
> > --runner_option=--blah=bar --runner_option=--foo=baz1
> --runner_option=--foo=baz2
>
> I'm -1 on this format. We should move away from the idea that options
> == flags (as that doesn't compose well with other libraries that do
> their own flags parsing). The ability to parse a set of flags into
> options is just a convenience that an author may (or may not) choose
> to use (e.g. when running pipelines a long-lived process like a
> service or a notebook, the command line flags are almost certainly not
> the right interface).
>
> > 2) specified multiple times, we drop the explicit flag
> > --runner_option=blah=bar --runner_option=foo=baz1
> --runner_option=foo=baz2
>
> This or (4) is my preference.
>
> > 3) we use a string which the runner can choose to interpret however they
> want (JSON/XML shown below)
> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
> >
> --runner_option='barbaz1baz2'
>
> This would make validation hard. Also, I think it makes sense for some
> runner options to be "shared" (parallelism") by convention, so letting
> it be a free-form string wouldn't allow different runners to inspect
> different bits.
>
> We should consider if we should use urns for namespacing, and
> assigning semantic meaning to strings, here.
>
> > 4) we use a string which must be a specific format such as JSON (allows
> the SDK to do simple validation):
> > --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>
> I like this in that at least some validation can be performed, and
> expectations of how to format richer types. On the other hand it gets
> a bit verbose, given that most (I'd imagine) options will be simple.
> As with normal options,
>
> --option1=value1 --option2=value2
>
> is shorthand for {"option1": value1, "option2": value2}.
>
> I lean to 4 the most. With 2, you run into issues of what does
--runner_option=foo=["a", "b"] --runner_option=foo=["c", "d"] mean?
Is it an error or list of lists or concatenated. Similar issues for map
types represented via JSON object {...}


> > I would strongly suggest that we go with the "fetch" approach, since
> this makes the set of options discoverable and helps users find errors much
> earlier in their pipeline.
>
> This seems like an advanced feature that SDKs may want to support, but
> I wouldn't want to require this complexity for bootstrapping an SDK.
>
> SDKs that are starting off wouldn't need to "fetch" options, they could
choose to not support runner options or they could choose to pass all
options through to the runner blindly. Fetching the options only provides
the SDK the ability to provide error checking upfront and useful error/help
messages.


> Regarding always keeping runner options separate, +1, though I'm not
> sure the line is always clear.
>
>
> > On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw 
> wrote:
> >>
> >> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels 
> wrote:
> >> >
> >> > I agree that the current approach breaks the pipeline options contract
> >> > because "unknown" options get parsed in the same way as options which
> >> > have been defined by the user.
> >>
> >> FWIW, I think we're already breaking this "contract." Unknown options
> >> are silently ignored; with this change we just change how we record
> >> them. It still feels a bit hacky though.
> >>
> >> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
> >> > true/false flags. We want to pass all types of pipeline options to the
> >> > Runner.
> >>
> >> Experiments is an arbitrary set of strings, which can be of the form
> >> "param=value" if that's useful. (Dataflow does this.) There is, again,
> >> no namespacing on the param names, but we could user urns or impose
> >> some other structure here.
> >>
> >> > How to solve this?
> >> >
> >> > 1) Add all 

Re: Vendoring vulnerable Guava (CVE-2018-10237)

2018-10-15 Thread Lukasz Cwik
For example, we vendor gRPC and it still depends on 20.0 in its latest
version (https://mvnrepository.com/artifact/io.grpc/grpc-core/1.15.1).

On Mon, Oct 15, 2018 at 2:10 PM Lukasz Cwik  wrote:

> 20.0 is a common version used by many of our dependencies, using 20.0 is
> least likely to cause classpath issues. Note that with Guava 22.0+, they
> have said they won't introduce backwards incompatible changes anymore so
> getting past 22.0 would mean we could just rely on using the latest at all
> times.
>
> I'm not sure the cost of upgrading our dependencies to be compatible with
> 22.0+ though.
>
> On Mon, Oct 15, 2018 at 11:11 AM Andrew Pilloud 
> wrote:
>
>> We vendor a known vulnerable version of Guava. The specific vulnerability
>> is low to no impact on Beam but it does potentially affect any server that
>> uses Java serialization with Beam on the classpath. Do we have a reason for
>> still being on Guava 20.0?
>>
>> https://github.com/google/guava/wiki/CVE-2018-10237
>>
>> Andrew
>>
>


Re: Vendoring vulnerable Guava (CVE-2018-10237)

2018-10-15 Thread Lukasz Cwik
20.0 is a common version used by many of our dependencies, using 20.0 is
least likely to cause classpath issues. Note that with Guava 22.0+, they
have said they won't introduce backwards incompatible changes anymore so
getting past 22.0 would mean we could just rely on using the latest at all
times.

I'm not sure the cost of upgrading our dependencies to be compatible with
22.0+ though.

On Mon, Oct 15, 2018 at 11:11 AM Andrew Pilloud  wrote:

> We vendor a known vulnerable version of Guava. The specific vulnerability
> is low to no impact on Beam but it does potentially affect any server that
> uses Java serialization with Beam on the classpath. Do we have a reason for
> still being on Guava 20.0?
>
> https://github.com/google/guava/wiki/CVE-2018-10237
>
> Andrew
>


Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Robert Bradshaw
On Mon, Oct 15, 2018 at 7:50 PM Lukasz Cwik  wrote:
>
> I agree with the sentiment for better error checking.
>
> We can try to make it such that the SDK can "fetch" the set of options that 
> the runner supports by making a call to the Job API. The API could return a 
> list of option names (descriptions for --help purposes and also potentially 
> the expected format) which would remove the worry around "unknown" options. 
> Yes I understand to be able to make the Job API call, we may need to parse 
> some options from the args parameters first and then parse the unknown 
> options after they are fetched.

This is an interesting idea, but seems it could get quite complicated.
E.g. for delegating runners, one would first read the options to
determine which runner to fetch the options from, which would then
return a set of options that possibly depends on the values of some of
its options...

> Alternatively, we can choose an explicit format upfront.
> To expand on the exact format for --runner_option=..., here are some 
> different ideas:
> 1) Specified multiple times, each one is an explicit flag
> --runner_option=--blah=bar --runner_option=--foo=baz1 
> --runner_option=--foo=baz2

I'm -1 on this format. We should move away from the idea that options
== flags (as that doesn't compose well with other libraries that do
their own flags parsing). The ability to parse a set of flags into
options is just a convenience that an author may (or may not) choose
to use (e.g. when running pipelines a long-lived process like a
service or a notebook, the command line flags are almost certainly not
the right interface).

> 2) specified multiple times, we drop the explicit flag
> --runner_option=blah=bar --runner_option=foo=baz1 --runner_option=foo=baz2

This or (4) is my preference.

> 3) we use a string which the runner can choose to interpret however they want 
> (JSON/XML shown below)
> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
> --runner_option='barbaz1baz2'

This would make validation hard. Also, I think it makes sense for some
runner options to be "shared" (parallelism") by convention, so letting
it be a free-form string wouldn't allow different runners to inspect
different bits.

We should consider if we should use urns for namespacing, and
assigning semantic meaning to strings, here.

> 4) we use a string which must be a specific format such as JSON (allows the 
> SDK to do simple validation):
> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'

I like this in that at least some validation can be performed, and
expectations of how to format richer types. On the other hand it gets
a bit verbose, given that most (I'd imagine) options will be simple.
As with normal options,

--option1=value1 --option2=value2

is shorthand for {"option1": value1, "option2": value2}.

> I would strongly suggest that we go with the "fetch" approach, since this 
> makes the set of options discoverable and helps users find errors much 
> earlier in their pipeline.

This seems like an advanced feature that SDKs may want to support, but
I wouldn't want to require this complexity for bootstrapping an SDK.

Regarding always keeping runner options separate, +1, though I'm not
sure the line is always clear.


> On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw  wrote:
>>
>> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels  wrote:
>> >
>> > I agree that the current approach breaks the pipeline options contract
>> > because "unknown" options get parsed in the same way as options which
>> > have been defined by the user.
>>
>> FWIW, I think we're already breaking this "contract." Unknown options
>> are silently ignored; with this change we just change how we record
>> them. It still feels a bit hacky though.
>>
>> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
>> > true/false flags. We want to pass all types of pipeline options to the
>> > Runner.
>>
>> Experiments is an arbitrary set of strings, which can be of the form
>> "param=value" if that's useful. (Dataflow does this.) There is, again,
>> no namespacing on the param names, but we could user urns or impose
>> some other structure here.
>>
>> > How to solve this?
>> >
>> > 1) Add all options of all Runners to each SDK
>> > We added some of the FlinkRunner options to the Python SDK but realized
>> > syncing is rather cumbersome in the long term. However, we want the most
>> > important options to be validated on the client side.
>>
>> I don't think this is sustainable in the long run. However, thinking
>> about this, in the worse case validation happens after construction
>> but before execution (as with much of our other validation) so it
>> isn't that bad.
>>
>> > 2) Pass "unknown" options via a separate list in the Proto which can
>> > only be accessed internally by the Runners. This still allows passing
>> > arbitrary options but we wouldn't leak unknown options and display them
>> > as top-level options.
>>
>> I think there needs 

Re: Fwd: Slack invitation

2018-10-15 Thread Lukasz Cwik
I got the the self service Slack invite link working again
https://s.apache.org/slack-invite by having it regenerated.

On Wed, Oct 10, 2018 at 2:13 PM Filip Popić  wrote:

> I got it, thank you!
>
> On Wed, 10 Oct 2018 at 16:17, Jean-Baptiste Onofré 
> wrote:
>
>> You didn't receive it ?
>>
>> Let me try another time.
>>
>> Regards
>> JB
>> Le 10 oct. 2018, à 17:15, "Filip Popić"  a écrit:
>>>
>>> Any news regarding invitation?
>>>
>>> On Mon, 8 Oct 2018 at 17:24, Jean-Baptiste Onofré < j...@nanthrax.net>
>>> wrote:
>>>
 Ok I will send it to you as well.

 Regards
 JB
 Le 8 oct. 2018, à 18:23, Emmanuel Bastien < o...@ebastien.name> a écrit:
>
> Hello,
> I would like to join the Beam Slack channel. Could someone send me an
> invitation?
> Thanks in advance!
> Emmanuel
>
>


[DISCUSS] GitHub comments to different list

2018-10-15 Thread Kenneth Knowles
Incidentally, on https://issues.apache.org/jira/browse/INFRA-17140 the
cause of our misconfiguration was that many projects ask to move GitHub
comments to a different list.

Just bringing it up for completeness, not advocating for it.

Would anyone choose to subscribe to this list? (I wouldn't)

WDYT?

Kenn


Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Ismaël Mejía
+1
On Mon, Oct 15, 2018 at 8:14 PM Mikhail Gryzykhin  wrote:
>
> +1 also suggested that in another thread.
>
> --Mikhail
>
> Have feedback?
>
>
> On Mon, Oct 15, 2018 at 11:10 AM Rui Wang  wrote:
>>
>> +1
>>
>> Agree that people might be only interested in JIRA activities but not in the 
>> build.
>>
>>
>> -Rui
>>
>> On Mon, Oct 15, 2018 at 10:27 AM Andrew Pilloud  wrote:
>>>
>>> +1. A bunch of Jenkins spam goes to dev as well. Would be good to it all to 
>>> a new list.
>>>
>>> Andrew
>>>
>>> On Mon, Oct 15, 2018 at 10:00 AM Colm O hEigeartaigh  
>>> wrote:

 As a rationale, some users might be interested in seeing JIRA activity but 
 might not care about whether the build is broken or not :-)

 Colm.

 On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles  wrote:
>
> Separating this out from the Jira notification thread.
>
> Colm suggests that we also separate build notifications.
>
> WDYT?
>
> Kenn



 --
 Colm O hEigeartaigh

 Talend Community Coder
 http://coders.talend.com


Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Thomas Weise
Would it be better to generally separate the runner options (whether they
are unknown or not) from other pipeline options?


On Mon, Oct 15, 2018 at 10:55 AM Lukasz Cwik  wrote:

> Note, that thinking ahead to cross language pipelines, we'll want
> something like "options" discovery as well. So reusing this concept for
> runners makes sense to me.
>
> On Mon, Oct 15, 2018 at 10:50 AM Lukasz Cwik  wrote:
>
>> I agree with the sentiment for better error checking.
>>
>> We can try to make it such that the SDK can "fetch" the set of options
>> that the runner supports by making a call to the Job API. The API could
>> return a list of option names (descriptions for --help purposes and also
>> potentially the expected format) which would remove the worry around
>> "unknown" options. Yes I understand to be able to make the Job API call, we
>> may need to parse some options from the args parameters first and then
>> parse the unknown options after they are fetched.
>>
>> Alternatively, we can choose an explicit format upfront.
>> To expand on the exact format for --runner_option=..., here are some
>> different ideas:
>> 1) Specified multiple times, each one is an explicit flag
>> --runner_option=--blah=bar --runner_option=--foo=baz1
>> --runner_option=--foo=baz2
>>
>> 2) specified multiple times, we drop the explicit flag
>> --runner_option=blah=bar --runner_option=foo=baz1
>> --runner_option=foo=baz2
>>
>> 3) we use a string which the runner can choose to interpret however they
>> want (JSON/XML shown below)
>> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>>
>> --runner_option='barbaz1baz2'
>>
>> 4) we use a string which must be a specific format such as JSON (allows
>> the SDK to do simple validation):
>> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>>
>> I would strongly suggest that we go with the "fetch" approach, since this
>> makes the set of options discoverable and helps users find errors much
>> earlier in their pipeline.
>>
>>
>>
>> On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw 
>> wrote:
>>
>>> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels 
>>> wrote:
>>> >
>>> > I agree that the current approach breaks the pipeline options contract
>>> > because "unknown" options get parsed in the same way as options which
>>> > have been defined by the user.
>>>
>>> FWIW, I think we're already breaking this "contract." Unknown options
>>> are silently ignored; with this change we just change how we record
>>> them. It still feels a bit hacky though.
>>>
>>> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
>>> > true/false flags. We want to pass all types of pipeline options to the
>>> > Runner.
>>>
>>> Experiments is an arbitrary set of strings, which can be of the form
>>> "param=value" if that's useful. (Dataflow does this.) There is, again,
>>> no namespacing on the param names, but we could user urns or impose
>>> some other structure here.
>>>
>>> > How to solve this?
>>> >
>>> > 1) Add all options of all Runners to each SDK
>>> > We added some of the FlinkRunner options to the Python SDK but realized
>>> > syncing is rather cumbersome in the long term. However, we want the
>>> most
>>> > important options to be validated on the client side.
>>>
>>> I don't think this is sustainable in the long run. However, thinking
>>> about this, in the worse case validation happens after construction
>>> but before execution (as with much of our other validation) so it
>>> isn't that bad.
>>>
>>> > 2) Pass "unknown" options via a separate list in the Proto which can
>>> > only be accessed internally by the Runners. This still allows passing
>>> > arbitrary options but we wouldn't leak unknown options and display them
>>> > as top-level options.
>>>
>>> I think there needs to be a way for the user to communicate values
>>> directly to the runner regardless of the SDK. My preference would be
>>> to make this explicit, e.g. (repeated) --runner_option=..., rather
>>> than scooping up all unknown flags at command line parsing time.
>>> Perhaps an SDK that is aware of some runners could choose to lift
>>> these as top-level options, but still pass them as runner options.
>>>
>>> > On 13.10.18 02:34, Charles Chen wrote:
>>> > > The current release branch
>>> > > (https://github.com/apache/beam/commits/release-2.8.0) was cut
>>> after the
>>> > > revert went in.  Sent out https://github.com/apache/beam/pull/6683
>>> as a
>>> > > revert of the revert.  Regarding your comment above, I can help out
>>> with
>>> > > the design / PR reviews for common Python code as you suggest.
>>> > >
>>> > > On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise >> > > > wrote:
>>> > >
>>> > > Thanks, will tag you and looking forward to feedback so we can
>>> > > ensure that changes work for everyone.
>>> > >
>>> > > Looking at the PR, I see agreement from Max to revert the change
>>> on
>>> > > the release branch, but not in master. Would you mind to restore
>>> 

Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Mikhail Gryzykhin
+1 also suggested that in another thread.

--Mikhail

Have feedback ?


On Mon, Oct 15, 2018 at 11:10 AM Rui Wang  wrote:

> +1
>
> Agree that people might be only interested in JIRA activities but not in
> the build.
>
>
> -Rui
>
> On Mon, Oct 15, 2018 at 10:27 AM Andrew Pilloud 
> wrote:
>
>> +1. A bunch of Jenkins spam goes to dev as well. Would be good to it all
>> to a new list.
>>
>> Andrew
>>
>> On Mon, Oct 15, 2018 at 10:00 AM Colm O hEigeartaigh 
>> wrote:
>>
>>> As a rationale, some users might be interested in seeing JIRA activity
>>> but might not care about whether the build is broken or not :-)
>>>
>>> Colm.
>>>
>>> On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles  wrote:
>>>
 Separating this out from the Jira notification thread.

 Colm suggests that we also separate build notifications.

 WDYT?

 Kenn

>>>
>>>
>>> --
>>> Colm O hEigeartaigh
>>>
>>> Talend Community Coder
>>> http://coders.talend.com
>>>
>>


Vendoring vulnerable Guava (CVE-2018-10237)

2018-10-15 Thread Andrew Pilloud
We vendor a known vulnerable version of Guava. The specific vulnerability
is low to no impact on Beam but it does potentially affect any server that
uses Java serialization with Beam on the classpath. Do we have a reason for
still being on Guava 20.0?

https://github.com/google/guava/wiki/CVE-2018-10237

Andrew


Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Rui Wang
+1

Agree that people might be only interested in JIRA activities but not in
the build.


-Rui

On Mon, Oct 15, 2018 at 10:27 AM Andrew Pilloud  wrote:

> +1. A bunch of Jenkins spam goes to dev as well. Would be good to it all
> to a new list.
>
> Andrew
>
> On Mon, Oct 15, 2018 at 10:00 AM Colm O hEigeartaigh 
> wrote:
>
>> As a rationale, some users might be interested in seeing JIRA activity
>> but might not care about whether the build is broken or not :-)
>>
>> Colm.
>>
>> On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles  wrote:
>>
>>> Separating this out from the Jira notification thread.
>>>
>>> Colm suggests that we also separate build notifications.
>>>
>>> WDYT?
>>>
>>> Kenn
>>>
>>
>>
>> --
>> Colm O hEigeartaigh
>>
>> Talend Community Coder
>> http://coders.talend.com
>>
>


Re: Beam Samza Runner status update

2018-10-15 Thread Lukasz Cwik
I'm excited to hear about making Samza capable of running non-Java
pipelines.

On Fri, Oct 12, 2018 at 9:46 AM Xinyu Liu  wrote:

> @Max: absolutely we should work together! FlinkRunner has been our best
> reference since the start of our SamzaRunner, and the previous work in
> Flink portable runner has been extremely valuable to us too. We haven't got
> to the point of portable stateful processing yet. Our next step is to hook
> up a streaming source, i.e. Kafka, and test out streaming capabilities such
> as watermarks, windowing and triggers. For us, reading from Kafka will
> happen on the Java side (we have quite a lot of extensions of Kafka in
> LinkedIn), so we will try to create some internal Python API and do the
> translation right now. On the other hand, we are following up with
> BEAM-2937 for the optimization of Combine in portable runner. Previously we
> run into some state problem without this being resolved. Anyway, look
> forward to syncing up with you more!
>
> Thanks,
> Xinyu
>
>
>
> On Fri, Oct 12, 2018 at 1:40 AM Maximilian Michels  wrote:
>
>> Thanks for the updating, Xinyu and Hai! Great to see another Running
>> emerging :)
>>
>> I'm on the FlinkRunner. Looking forward to working together with you to
>> make the Beam Runners even better. Particularly, we should sync on the
>> portability, as some things are still to be fleshed out. In Flink, we
>> are starting to integrate portable State.
>>
>> Best,
>> Max
>>
>> On 11.10.18 05:14, Jesse Anderson wrote:
>> > Interesting
>> >
>> > On Wed, Oct 10, 2018, 3:49 PM Kenneth Knowles > > > wrote:
>> >
>> > Welcome, Hai!
>> >
>> > On Wed, Oct 10, 2018 at 3:46 PM Hai Lu > > > wrote:
>> >
>> > Hi, all
>> >
>> > This is Hai from LinkedIn. As Xinyu mentioned, I have been
>> > working on portable API for Samza runner and made some solid
>> > progress. It's been a very smooth process (although not
>> > effortless for sure) and I'm really grateful for the great
>> > platform that you all have built. I'm very impressed. Bravo!
>> >
>> > Excited to work with everyone on Beam. Do expect more questions
>> > from me down the road.
>> >
>> > Thanks,
>> > Hai
>> >
>> > On Wed, Oct 10, 2018 at 12:36 PM Kenneth Knowles
>> > mailto:k...@apache.org>> wrote:
>> >
>> > Clarification: Thomas Groh wrote the fuser, not me!
>> >
>> > Thanks for the sharing all this. Really cool.
>> >
>> > Kenn
>> >
>> > On Wed, Oct 10, 2018 at 11:17 AM Rui Wang <
>> ruw...@google.com
>> > > wrote:
>> >
>> > Thanks for sharing! it's so exciting to hear that Beam
>> > is being used on Samza in production @LinkedIn! Your
>> > feedback will be helpful to Beam community!
>> >
>> > Besides, Beam supports SQL right now and hopefully Beam
>> > community could also receive feedback on BeamSQL
>> > <
>> https://beam.apache.org/documentation/dsls/sql/overview/> in
>> > the future.
>> >
>> > -Rui
>> >
>> > On Wed, Oct 10, 2018 at 11:10 AM Jean-Baptiste Onofré
>> > mailto:j...@nanthrax.net>> wrote:
>> >
>> > Thanks for sharing and congrats for this great work
>> !
>> >
>> > Regards
>> > JB
>> > Le 10 oct. 2018, à 20:23, Xinyu Liu > > @gmail.com 
>> > target=_blank>xinyuliu.us
>> > @gmail.com >
>> a
>> > écrit:
>> >
>> > Hi, All,
>> >
>> > It's been over four months since we added the
>> > Samza Runner to Beam, and we've been making a
>> > lot of progress after that. Here I would like to
>> > update your guys and share some really good news
>> > happening here at LinkedIn:
>> >
>> > 1) First Beam job in production @LInkedIn!
>> > After a few rounds of testing and benchmarking,
>> > we finally rolled out our first Beam job here!
>> > The job uses quite a few features, such as event
>> > time, fixed/session windowing, early triggering,
>> > and stateful processing. Our first customer is
>> > very happy and they highly appraise the
>> > easy-to-use Beam API as well as powerful
>> > processing model. Due to the limited resources
>> > here, we put our full trust in the work you guys
>> >   

Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Lukasz Cwik
Note, that thinking ahead to cross language pipelines, we'll want something
like "options" discovery as well. So reusing this concept for runners makes
sense to me.

On Mon, Oct 15, 2018 at 10:50 AM Lukasz Cwik  wrote:

> I agree with the sentiment for better error checking.
>
> We can try to make it such that the SDK can "fetch" the set of options
> that the runner supports by making a call to the Job API. The API could
> return a list of option names (descriptions for --help purposes and also
> potentially the expected format) which would remove the worry around
> "unknown" options. Yes I understand to be able to make the Job API call, we
> may need to parse some options from the args parameters first and then
> parse the unknown options after they are fetched.
>
> Alternatively, we can choose an explicit format upfront.
> To expand on the exact format for --runner_option=..., here are some
> different ideas:
> 1) Specified multiple times, each one is an explicit flag
> --runner_option=--blah=bar --runner_option=--foo=baz1
> --runner_option=--foo=baz2
>
> 2) specified multiple times, we drop the explicit flag
> --runner_option=blah=bar --runner_option=foo=baz1 --runner_option=foo=baz2
>
> 3) we use a string which the runner can choose to interpret however they
> want (JSON/XML shown below)
> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>
> --runner_option='barbaz1baz2'
>
> 4) we use a string which must be a specific format such as JSON (allows
> the SDK to do simple validation):
> --runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
>
> I would strongly suggest that we go with the "fetch" approach, since this
> makes the set of options discoverable and helps users find errors much
> earlier in their pipeline.
>
>
>
> On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw 
> wrote:
>
>> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels 
>> wrote:
>> >
>> > I agree that the current approach breaks the pipeline options contract
>> > because "unknown" options get parsed in the same way as options which
>> > have been defined by the user.
>>
>> FWIW, I think we're already breaking this "contract." Unknown options
>> are silently ignored; with this change we just change how we record
>> them. It still feels a bit hacky though.
>>
>> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
>> > true/false flags. We want to pass all types of pipeline options to the
>> > Runner.
>>
>> Experiments is an arbitrary set of strings, which can be of the form
>> "param=value" if that's useful. (Dataflow does this.) There is, again,
>> no namespacing on the param names, but we could user urns or impose
>> some other structure here.
>>
>> > How to solve this?
>> >
>> > 1) Add all options of all Runners to each SDK
>> > We added some of the FlinkRunner options to the Python SDK but realized
>> > syncing is rather cumbersome in the long term. However, we want the most
>> > important options to be validated on the client side.
>>
>> I don't think this is sustainable in the long run. However, thinking
>> about this, in the worse case validation happens after construction
>> but before execution (as with much of our other validation) so it
>> isn't that bad.
>>
>> > 2) Pass "unknown" options via a separate list in the Proto which can
>> > only be accessed internally by the Runners. This still allows passing
>> > arbitrary options but we wouldn't leak unknown options and display them
>> > as top-level options.
>>
>> I think there needs to be a way for the user to communicate values
>> directly to the runner regardless of the SDK. My preference would be
>> to make this explicit, e.g. (repeated) --runner_option=..., rather
>> than scooping up all unknown flags at command line parsing time.
>> Perhaps an SDK that is aware of some runners could choose to lift
>> these as top-level options, but still pass them as runner options.
>>
>> > On 13.10.18 02:34, Charles Chen wrote:
>> > > The current release branch
>> > > (https://github.com/apache/beam/commits/release-2.8.0) was cut after
>> the
>> > > revert went in.  Sent out https://github.com/apache/beam/pull/6683
>> as a
>> > > revert of the revert.  Regarding your comment above, I can help out
>> with
>> > > the design / PR reviews for common Python code as you suggest.
>> > >
>> > > On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise > > > > wrote:
>> > >
>> > > Thanks, will tag you and looking forward to feedback so we can
>> > > ensure that changes work for everyone.
>> > >
>> > > Looking at the PR, I see agreement from Max to revert the change
>> on
>> > > the release branch, but not in master. Would you mind to restore
>> it
>> > > in master?
>> > >
>> > > Thanks
>> > >
>> > > On Fri, Oct 12, 2018 at 4:40 PM Ahmet Altay > > > > wrote:
>> > >
>> > >
>> > >
>> > > On Fri, Oct 12, 2018 at 11:31 AM, Charles Chen <
>> c...@google.com
>> > > 

Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Lukasz Cwik
I agree with the sentiment for better error checking.

We can try to make it such that the SDK can "fetch" the set of options that
the runner supports by making a call to the Job API. The API could return a
list of option names (descriptions for --help purposes and also potentially
the expected format) which would remove the worry around "unknown" options.
Yes I understand to be able to make the Job API call, we may need to parse
some options from the args parameters first and then parse the unknown
options after they are fetched.

Alternatively, we can choose an explicit format upfront.
To expand on the exact format for --runner_option=..., here are some
different ideas:
1) Specified multiple times, each one is an explicit flag
--runner_option=--blah=bar --runner_option=--foo=baz1
--runner_option=--foo=baz2

2) specified multiple times, we drop the explicit flag
--runner_option=blah=bar --runner_option=foo=baz1 --runner_option=foo=baz2

3) we use a string which the runner can choose to interpret however they
want (JSON/XML shown below)
--runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'
--runner_option='barbaz1baz2'

4) we use a string which must be a specific format such as JSON (allows the
SDK to do simple validation):
--runner_option='{"blah": "bar", "foo": ["baz1", "baz2"]}'

I would strongly suggest that we go with the "fetch" approach, since this
makes the set of options discoverable and helps users find errors much
earlier in their pipeline.



On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw  wrote:

> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels  wrote:
> >
> > I agree that the current approach breaks the pipeline options contract
> > because "unknown" options get parsed in the same way as options which
> > have been defined by the user.
>
> FWIW, I think we're already breaking this "contract." Unknown options
> are silently ignored; with this change we just change how we record
> them. It still feels a bit hacky though.
>
> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
> > true/false flags. We want to pass all types of pipeline options to the
> > Runner.
>
> Experiments is an arbitrary set of strings, which can be of the form
> "param=value" if that's useful. (Dataflow does this.) There is, again,
> no namespacing on the param names, but we could user urns or impose
> some other structure here.
>
> > How to solve this?
> >
> > 1) Add all options of all Runners to each SDK
> > We added some of the FlinkRunner options to the Python SDK but realized
> > syncing is rather cumbersome in the long term. However, we want the most
> > important options to be validated on the client side.
>
> I don't think this is sustainable in the long run. However, thinking
> about this, in the worse case validation happens after construction
> but before execution (as with much of our other validation) so it
> isn't that bad.
>
> > 2) Pass "unknown" options via a separate list in the Proto which can
> > only be accessed internally by the Runners. This still allows passing
> > arbitrary options but we wouldn't leak unknown options and display them
> > as top-level options.
>
> I think there needs to be a way for the user to communicate values
> directly to the runner regardless of the SDK. My preference would be
> to make this explicit, e.g. (repeated) --runner_option=..., rather
> than scooping up all unknown flags at command line parsing time.
> Perhaps an SDK that is aware of some runners could choose to lift
> these as top-level options, but still pass them as runner options.
>
> > On 13.10.18 02:34, Charles Chen wrote:
> > > The current release branch
> > > (https://github.com/apache/beam/commits/release-2.8.0) was cut after
> the
> > > revert went in.  Sent out https://github.com/apache/beam/pull/6683 as
> a
> > > revert of the revert.  Regarding your comment above, I can help out
> with
> > > the design / PR reviews for common Python code as you suggest.
> > >
> > > On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise  > > > wrote:
> > >
> > > Thanks, will tag you and looking forward to feedback so we can
> > > ensure that changes work for everyone.
> > >
> > > Looking at the PR, I see agreement from Max to revert the change on
> > > the release branch, but not in master. Would you mind to restore it
> > > in master?
> > >
> > > Thanks
> > >
> > > On Fri, Oct 12, 2018 at 4:40 PM Ahmet Altay  > > > wrote:
> > >
> > >
> > >
> > > On Fri, Oct 12, 2018 at 11:31 AM, Charles Chen  > > > wrote:
> > >
> > > What I mean is that a user may find that it works for them
> > > to pass "--myarg blah" and access it as "options.myarg"
> > > without explicitly defining a "my_arg" flag due to the
> added
> > > logic.  This is not the intended behavior and we may want
> to
> > > change this implementation detail in the 

Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Udi Meiri
+1 for explicit --runner_option=param=val,...
It's hard to tell otherwise where an option is going to,

On Mon, Oct 15, 2018 at 8:04 AM Robert Bradshaw  wrote:

> On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels  wrote:
> >
> > I agree that the current approach breaks the pipeline options contract
> > because "unknown" options get parsed in the same way as options which
> > have been defined by the user.
>
> FWIW, I think we're already breaking this "contract." Unknown options
> are silently ignored; with this change we just change how we record
> them. It still feels a bit hacky though.
>
> > I'm not sure the `experiments` flag works for us. AFAIK it only allows
> > true/false flags. We want to pass all types of pipeline options to the
> > Runner.
>
> Experiments is an arbitrary set of strings, which can be of the form
> "param=value" if that's useful. (Dataflow does this.) There is, again,
> no namespacing on the param names, but we could user urns or impose
> some other structure here.
>
> > How to solve this?
> >
> > 1) Add all options of all Runners to each SDK
> > We added some of the FlinkRunner options to the Python SDK but realized
> > syncing is rather cumbersome in the long term. However, we want the most
> > important options to be validated on the client side.
>
> I don't think this is sustainable in the long run. However, thinking
> about this, in the worse case validation happens after construction
> but before execution (as with much of our other validation) so it
> isn't that bad.
>
> > 2) Pass "unknown" options via a separate list in the Proto which can
> > only be accessed internally by the Runners. This still allows passing
> > arbitrary options but we wouldn't leak unknown options and display them
> > as top-level options.
>
> I think there needs to be a way for the user to communicate values
> directly to the runner regardless of the SDK. My preference would be
> to make this explicit, e.g. (repeated) --runner_option=..., rather
> than scooping up all unknown flags at command line parsing time.
> Perhaps an SDK that is aware of some runners could choose to lift
> these as top-level options, but still pass them as runner options.
>
> > On 13.10.18 02:34, Charles Chen wrote:
> > > The current release branch
> > > (https://github.com/apache/beam/commits/release-2.8.0) was cut after
> the
> > > revert went in.  Sent out https://github.com/apache/beam/pull/6683 as
> a
> > > revert of the revert.  Regarding your comment above, I can help out
> with
> > > the design / PR reviews for common Python code as you suggest.
> > >
> > > On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise  > > > wrote:
> > >
> > > Thanks, will tag you and looking forward to feedback so we can
> > > ensure that changes work for everyone.
> > >
> > > Looking at the PR, I see agreement from Max to revert the change on
> > > the release branch, but not in master. Would you mind to restore it
> > > in master?
> > >
> > > Thanks
> > >
> > > On Fri, Oct 12, 2018 at 4:40 PM Ahmet Altay  > > > wrote:
> > >
> > >
> > >
> > > On Fri, Oct 12, 2018 at 11:31 AM, Charles Chen  > > > wrote:
> > >
> > > What I mean is that a user may find that it works for them
> > > to pass "--myarg blah" and access it as "options.myarg"
> > > without explicitly defining a "my_arg" flag due to the
> added
> > > logic.  This is not the intended behavior and we may want
> to
> > > change this implementation detail in the future.  However,
> > > having this logic in a released version makes it hard to
> > > change this behavior since users may erroneously depend on
> > > this undocumented behavior.  Instead, we should namespace /
> > > scope this so that it is obvious that this is meant for
> > > runner (and not Beam user) consumption.
> > >
> > > On Fri, Oct 12, 2018 at 10:48 AM Thomas Weise
> > > mailto:t...@apache.org>> wrote:
> > >
> > > Can you please elaborate more what practical problems
> > > this introduces for users?
> > >
> > > I can see that this change allows a user to specify a
> > > runner specific option, which in the future may change
> > > because we decide to scope differently. If this only
> > > affects users of the portable Flink runner (like us),
> > > then no need to revert, because at this early stage we
> > > prefer something that works over being blocked.
> > >
> > > It would also be really great if some of the core
> Python
> > > SDK developers could help out with the design aspects
> > > and PR reviews of changes that affect common Python
> > > code. Anyone who specifically wants 

Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Andrew Pilloud
+1. A bunch of Jenkins spam goes to dev as well. Would be good to it all to
a new list.

Andrew

On Mon, Oct 15, 2018 at 10:00 AM Colm O hEigeartaigh 
wrote:

> As a rationale, some users might be interested in seeing JIRA activity but
> might not care about whether the build is broken or not :-)
>
> Colm.
>
> On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles  wrote:
>
>> Separating this out from the Jira notification thread.
>>
>> Colm suggests that we also separate build notifications.
>>
>> WDYT?
>>
>> Kenn
>>
>
>
> --
> Colm O hEigeartaigh
>
> Talend Community Coder
> http://coders.talend.com
>


Re: [DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Colm O hEigeartaigh
As a rationale, some users might be interested in seeing JIRA activity but
might not care about whether the build is broken or not :-)

Colm.

On Mon, Oct 15, 2018 at 5:56 PM Kenneth Knowles  wrote:

> Separating this out from the Jira notification thread.
>
> Colm suggests that we also separate build notifications.
>
> WDYT?
>
> Kenn
>


-- 
Colm O hEigeartaigh

Talend Community Coder
http://coders.talend.com


[DISCUSS] Separate Jenkins notifications to a new mailing list

2018-10-15 Thread Kenneth Knowles
Separating this out from the Jira notification thread.

Colm suggests that we also separate build notifications.

WDYT?

Kenn


Re: [DISCUSS] - Separate JIRA notifications to a new mailing list

2018-10-15 Thread Kenneth Knowles
The list exists and I'm working with infra to get the configuration right
(first try had some hiccups).

Colm: I like that idea as well. This thread is a bit long and it might get
buried. I'll just start another to be extra clear.

Kenn

On Mon, Oct 15, 2018 at 6:42 AM Colm O hEigeartaigh 
wrote:

> Excellent, thanks everyone! I am also +1 to the suggestion made in the
> thread to separate out build mails into a separate mailing list as well.
>
> Colm.
>
> On Mon, Oct 15, 2018 at 2:18 PM Maximilian Michels  wrote:
>
>> The list has been created. You can already subscribe to the list by
>> sending an empty mail to: issues-subscr...@beam.apache.org
>>
>> Could we announce the time when we plan to make the switch to the new
>> list? Also, the mailing list page needs to be updated:
>> https://beam.apache.org/community/contact-us/
>>
>> On 11.10.18 23:21, Rui Wang wrote:
>> > +1
>> >
>> > -Rui
>> >
>> > On Thu, Oct 11, 2018 at 12:53 PM Ankur Goenka > > > wrote:
>> >
>> > +1
>> >
>> > On Thu, Oct 11, 2018 at 12:14 PM Suneel Marthi
>> > mailto:suneel.mar...@gmail.com>> wrote:
>> >
>> > +1
>> >
>> > Sent from my iPhone
>> >
>> > On Oct 11, 2018, at 8:03 PM, Łukasz Gajowy
>> > mailto:lukasz.gaj...@gmail.com>>
>> wrote:
>> >
>> >> This is a good idea. +1
>> >>
>> >> Łukasz
>> >>
>> >>
>> >> czw., 11 paź 2018, 18:01 użytkownik Udi Meiri
>> >> mailto:eh...@google.com>> napisał:
>> >>
>> >> +1 to split JIRA notifications
>> >>
>> >> On Thu, Oct 11, 2018 at 9:13 AM Kenneth Knowles
>> >> mailto:k...@apache.org>> wrote:
>> >>
>> >>
>> >> On Thu, Oct 11, 2018 at 9:10 AM Mikhail Gryzykhin
>> >> > >> > wrote:
>> >>
>> >> +1.
>> >> Should we separate Jenkins notifications as well?
>> >>
>> >>
>> >> I'm worried this question will get buried in the
>> >> thread. Would you mind separating it into another
>> >> thread if you would like to discuss?
>> >>
>> >> Kenn
>> >>
>> >> On Thu, Oct 11, 2018, 08:59 Scott Wegner
>> >> mailto:sc...@apache.org> wrote:
>> >>
>> >> +1, commits@ is too noisy to be useful
>> currently.
>> >>
>> >> On Thu, Oct 11, 2018 at 8:04 AM Maximilian
>> >> Michels > >> > wrote:
>> >>
>> >> +1
>> >>
>> >> I guess most people have already filters
>> >> in place to separate commits
>> >> and JIRA issues. JIRA really has nothing
>> >> to do in the commits list.
>> >>
>> >> On 11.10.18 15:53, Kenneth Knowles wrote:
>> >> > +1
>> >> >
>> >> > I've suggested the same. Canonical.
>> >> >
>> >> > On Thu, Oct 11, 2018, 06:19 Thomas Weise
>> >> mailto:t...@apache.org>
>> >> > > >> >> wrote:
>> >> >
>> >> > +1
>> >> >
>> >> >
>> >> > On Thu, Oct 11, 2018 at 6:18 AM
>> >> Etienne Chauchot
>> >> > > >> 
>> >> > >> >> wrote:
>> >> >
>> >> > +1 for me also, my gmail filters
>> >> list is kind of overflowed :)
>> >> >
>> >> > Etienne
>> >> >
>> >> > Le jeudi 11 octobre 2018 à 14:44
>> >> +0200, Robert Bradshaw a écrit :
>> >> >> Huge +1 from me too.
>> >> >> On Thu, Oct 11, 2018 at 2:42 PM
>> >> Jean-Baptiste Onofré > >> 
>> >> > >> >> wrote:
>> >> >>
>> >> >> +1
>> >> >>
>> >> >> We are doing the same in Karaf
>> >> 

[ANNOUNCE] New committers & PMC members, Summer 2018 edition

2018-10-15 Thread Kenneth Knowles
Hi all,

Since our last announcement in May, we have added many more committers and
a new PMC member. Some of these may have been in the monthly newsletter or
mentioned elsewhere, but I wanted to be sure to have a loud announcement on
the list directly.

Please join me in belatedly welcoming...

New PMC member: Thomas Weise
 - Author of the ApexRunner, the first additional runner after Beam began
incubation.
 - Recently heavily involved in Python-on-Flink efforts.
 - Outside his contributions to Beam, Thomas is PMC chair for Apache Apex.

New committers:

 - Charles Chen, longtime contributor to Python SDK, Python direct runner,
state & timers
 - Łukasz  Gajowy, testing infrastructure, benchmarks, build system
improvements
 - Anton Kedin, contributor to SQL and schemas, helper on StackOverflow
 - Andrew Pilloud, contributor to SQL, very active on dev@, infra and
release help
 - Tim Robertson, contributor to many IOs, major code health work
 - Alexey Romanenko, contributor to many IOs, Nexmark benchmarks
 - Henning Rohde, contributor to Go SDK, incl. ip fun, and portability
protos and design
 - Scott Wegner, one of our longest contributors, major infra improvements

And while I've noted some areas of contribution for each, most importantly
everyone on this list is a valued member of the Beam community that the PMC
trusts with the responsibilities of a Beam committer [1].

A big thanks to all for their contributions.

Kenn

[1]
https://beam.apache.org/contribute/become-a-committer/#an-apache-beam-committer


Re: PBegin

2018-10-15 Thread Chamikara Jayalath
MongoDBIO is based on BoundedSource framework so there's no easy way to
introduce custom code (a ParDo) that precede it in a pipeline. A ReadAll
transform (as JB mentioned) will be ParDo based and you will be able to
have a preceding custom ParDo that runs the initialization and feeds data
into the source. So I agree that this will be the proper solution.

Downside is that some advanced features (for example, dynamic work
rebalancing) will not be supported till Splittable DoFn is fully fleshed
out. But looks like MongoDB currently does not support this feature anyways
so it should be OK.

Thanks,
Cham

On Mon, Oct 15, 2018 at 7:08 AM Jean-Baptiste Onofré 
wrote:

> JdbcIO uses the following:
>
>   return input
>   .apply(Create.of((Void) null))
>   .apply(
>   JdbcIO.readAll()
>
> .withDataSourceConfiguration(getDataSourceConfiguration())
>   .withQuery(getQuery())
>   .withCoder(getCoder())
>   .withRowMapper(getRowMapper())
>   .withFetchSize(getFetchSize())
>   .withParameterSetter(
>   (element, preparedStatement) -> {
> if (getStatementPreparator() != null) {
>
> getStatementPreparator().setParameters(preparedStatement);
> }
>   }));
>
> You can see that PBegin triggers readAll() that actually fires the
> configuration and fetching.
>
> I think we can do the same in MongoDbIO.
>
> Regards
> JB
>
> On 15/10/2018 16:00, Chaim Turkel wrote:
> > what would be the implementation for the JdbcIO?
> > On Mon, Oct 15, 2018 at 2:47 PM Jean-Baptiste Onofré 
> wrote:
> >>
> >> If you want to reuse MongoDbIO, there's no easy way.
> >>
> >> However, I can introduce the same change we did in Jdbc or Elasticsearch
> >> IOs.
> >>
> >> Agree ?
> >>
> >> Regards
> >> JB
> >>
> >> On 15/10/2018 13:46, Chaim Turkel wrote:
> >>> Thanks,
> >>>   I need to wrap MongoDbIO.read, and don't see an easy way to do it
> >>> chaim
> >>> On Mon, Oct 15, 2018 at 2:30 PM Jean-Baptiste Onofré 
> wrote:
> 
>  Hi Chaim,
> 
>  you can take a look on JdbcIO.
> 
>  You can create any "startup" PCollection on the PBegin, and then you
> can
>  can the DoFn based on that.
> 
>  Regards
>  JB
> 
>  On 15/10/2018 13:00, Chaim Turkel wrote:
> > Hi,
> >   I there a way to write code before the PBegin.
> > I am writeing a pipeline to connect to mongo with self signed ssl. I
> > need to init the ssl connection of the java before the mongo code. So
> > i need to write code before the PBegin but for it to run on each
> node?
> >
> >
> > Chaim
> >
> 
>  --
>  Jean-Baptiste Onofré
>  jbono...@apache.org
>  http://blog.nanthrax.net
>  Talend - http://www.talend.com
> >>>
> >>
> >> --
> >> Jean-Baptiste Onofré
> >> jbono...@apache.org
> >> http://blog.nanthrax.net
> >> Talend - http://www.talend.com
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>


Re: [DISCUSS] Committer Guidelines / Hygene before merging PRs

2018-10-15 Thread Thomas Weise
Here is my attempt to summarize the discussion, please see the TBDs.

I would work on a PR with respective contributor and committer guideline
updates next.

Thanks,
Thomas


Goals:

- Clear history with purpose and origin of changes
- Ability to perform a granular rollback, if necessary
- Efficiency and quality of review (avoid tiny or out-of-context changes or
huge mega-changes)
- Efficiency of authoring (don't want to wait on a review for a tiny bit
because GitHub makes it very hard to stack up reviews in sequence / don't
want to have major changes blocked because of difficulty of review)
- Ease of new contribution ("OK for committers to do more, while
new/one-time contributors shouldn't need to know or obey any policy"). TBD:
I think that quote needs clarification. IMO contributors are expected to
read and adhere to contribution guidelines.

Clean history:

- Commit messages should tag JIRAs and be otherwise descriptive. It should
not be necessary to find a merge or first PR commit to find out what caused
a change
- We want to squash "Fixup!", "Address comments" type of commits to achieve
a clean history
- We prefer that PR authors squash such commits after review is complete.
This expectation should be described clearly in the Contributor's Guide.
- The process should not burden committer due to back and forth with author
and deal with cleaning up PR history and other cosmetics. We want to reduce
committer overhead. But note that we don't want to shift the burden to
first time contributors either.
- Committer can use the "squash and merge" option (or modify the PR commits
in other ways). This should address the overhead concern. TBD: I would
suggest that the author, when this is explicitly not desired, needs to
indicate it upfront.
- Committer is ultimately responsible (committer guidelines) and we "trust
the committer's judgment"

Granularity of changes:

- We prefer small independent, incremental PRs with descriptive, isolated
commits. Each commit is a single clear change.
- It is OK to keep separate commits for different logical pieces of the
code, which make reviewing and revisiting code easier
- Making commits isolated is a good practice, PR author should be able to
relatively easily split the PR upon reviewer's request
- For multiple commits in a PR, every commit that gets merged should
compile and pass tests

Git playbook

- How to rollback multiple commits from single PR
- Simple step-by-step instructions for authors to cleanup history



On Mon, Oct 1, 2018 at 11:50 AM Rui Wang  wrote:

> +1 to add JIRA issue as the part of commit message. it requires less
> effort but will help a lot on our commit history. I used to not do that but
> I will start to add JIRA info to my future commits.
>
> -Rui
>
> On Mon, Oct 1, 2018 at 2:59 AM Alexey Romanenko 
> wrote:
>
>> +1 to what Anton said, I’m fully agree with this.
>>
>> Looking on the current commit history we can notice that there are many
>> commits like “fix”, “add/edit/remove”, “clean up”, etc, that, seems, were
>> introduced after the review process iterations. I think this makes commit
>> history too much verbose and more difficult to follow.
>>
>> Also, most of such commits don’t have Jira issue name as part of commit
>> message. So it requires to find a merge or fist PR commit in case we want
>> to find out what caused this change.
>>
>> I believe we can improve our commit guidelines in this way and it should
>> help to have commit history more clean and easy to read.
>>
>> On 1 Oct 2018, at 06:34, Kenneth Knowles  wrote:
>>
>> SGTM. I generally favor "trust the committer's judgment" aka (2)(d) when
>> it is obvious. We had some small communication problem about this before so
>> I just wanted to be careful to ask the author when it is not obvious.
>>
>> Kenn
>>
>> On Sat, Sep 29, 2018 at 9:06 AM Robert Bradshaw 
>> wrote:
>>
>>> There's two separate topics of discussion going on here.
>>>
>>> (1) Is it acceptable for PRs to have more than one commit. We've
>>> discussed this before, and the consensus seems to be that this is both
>>> allowed and intentional.
>>>
>>> (2) What to do about PRs that are clearly "[BEAM-] Implement
>>> feature" + (fixup/address comments/lint/...)*. I think this is easier (or
>>> at least quicker) to make a call on. Our options are
>>>
>>> (a) Merge as is.
>>> (b) Ask the author to do a final squash. (Doing a squash before an
>>> approval, however, makes reviewing more difficult, so this enforces another
>>> round.)
>>> (c) Having the committer manually pull, squash, (force push to PR?), and
>>> merge.
>>> (d) Use the "squash and merge" button in this case.
>>>
>>> Clearly, we shouldn't be doing any squashing unless the intent is
>>> obvious, but when it is, I think (d) is the sanest and safest route.
>>>
>>> - Robert
>>>
>>>
>>>
>>> On Fri, Sep 28, 2018 at 9:53 PM Kenneth Knowles  wrote:
>>>
 On Fri, Sep 28, 2018 at 10:29 AM Thomas Weise  wrote:

> +1 for stating the goal of clear provenance 

Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Robert Bradshaw
On Mon, Oct 15, 2018 at 3:58 PM Maximilian Michels  wrote:
>
> I agree that the current approach breaks the pipeline options contract
> because "unknown" options get parsed in the same way as options which
> have been defined by the user.

FWIW, I think we're already breaking this "contract." Unknown options
are silently ignored; with this change we just change how we record
them. It still feels a bit hacky though.

> I'm not sure the `experiments` flag works for us. AFAIK it only allows
> true/false flags. We want to pass all types of pipeline options to the
> Runner.

Experiments is an arbitrary set of strings, which can be of the form
"param=value" if that's useful. (Dataflow does this.) There is, again,
no namespacing on the param names, but we could user urns or impose
some other structure here.

> How to solve this?
>
> 1) Add all options of all Runners to each SDK
> We added some of the FlinkRunner options to the Python SDK but realized
> syncing is rather cumbersome in the long term. However, we want the most
> important options to be validated on the client side.

I don't think this is sustainable in the long run. However, thinking
about this, in the worse case validation happens after construction
but before execution (as with much of our other validation) so it
isn't that bad.

> 2) Pass "unknown" options via a separate list in the Proto which can
> only be accessed internally by the Runners. This still allows passing
> arbitrary options but we wouldn't leak unknown options and display them
> as top-level options.

I think there needs to be a way for the user to communicate values
directly to the runner regardless of the SDK. My preference would be
to make this explicit, e.g. (repeated) --runner_option=..., rather
than scooping up all unknown flags at command line parsing time.
Perhaps an SDK that is aware of some runners could choose to lift
these as top-level options, but still pass them as runner options.

> On 13.10.18 02:34, Charles Chen wrote:
> > The current release branch
> > (https://github.com/apache/beam/commits/release-2.8.0) was cut after the
> > revert went in.  Sent out https://github.com/apache/beam/pull/6683 as a
> > revert of the revert.  Regarding your comment above, I can help out with
> > the design / PR reviews for common Python code as you suggest.
> >
> > On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise  > > wrote:
> >
> > Thanks, will tag you and looking forward to feedback so we can
> > ensure that changes work for everyone.
> >
> > Looking at the PR, I see agreement from Max to revert the change on
> > the release branch, but not in master. Would you mind to restore it
> > in master?
> >
> > Thanks
> >
> > On Fri, Oct 12, 2018 at 4:40 PM Ahmet Altay  > > wrote:
> >
> >
> >
> > On Fri, Oct 12, 2018 at 11:31 AM, Charles Chen  > > wrote:
> >
> > What I mean is that a user may find that it works for them
> > to pass "--myarg blah" and access it as "options.myarg"
> > without explicitly defining a "my_arg" flag due to the added
> > logic.  This is not the intended behavior and we may want to
> > change this implementation detail in the future.  However,
> > having this logic in a released version makes it hard to
> > change this behavior since users may erroneously depend on
> > this undocumented behavior.  Instead, we should namespace /
> > scope this so that it is obvious that this is meant for
> > runner (and not Beam user) consumption.
> >
> > On Fri, Oct 12, 2018 at 10:48 AM Thomas Weise
> > mailto:t...@apache.org>> wrote:
> >
> > Can you please elaborate more what practical problems
> > this introduces for users?
> >
> > I can see that this change allows a user to specify a
> > runner specific option, which in the future may change
> > because we decide to scope differently. If this only
> > affects users of the portable Flink runner (like us),
> > then no need to revert, because at this early stage we
> > prefer something that works over being blocked.
> >
> > It would also be really great if some of the core Python
> > SDK developers could help out with the design aspects
> > and PR reviews of changes that affect common Python
> > code. Anyone who specifically wants to be tagged on
> > relevant JIRAs and PRs?
> >
> >
> > I would be happy to be tagged, and I can also help with
> > including other relevant folks whenever possible. In general I
> > think Robert, Charles, myself are good candidates.
> >
> >
> > Thanks
> >
> >
> > On Fri, Oct 12, 2018 at 

Re: PBegin

2018-10-15 Thread Jean-Baptiste Onofré
JdbcIO uses the following:

  return input
  .apply(Create.of((Void) null))
  .apply(
  JdbcIO.readAll()
  .withDataSourceConfiguration(getDataSourceConfiguration())
  .withQuery(getQuery())
  .withCoder(getCoder())
  .withRowMapper(getRowMapper())
  .withFetchSize(getFetchSize())
  .withParameterSetter(
  (element, preparedStatement) -> {
if (getStatementPreparator() != null) {

getStatementPreparator().setParameters(preparedStatement);
}
  }));

You can see that PBegin triggers readAll() that actually fires the
configuration and fetching.

I think we can do the same in MongoDbIO.

Regards
JB

On 15/10/2018 16:00, Chaim Turkel wrote:
> what would be the implementation for the JdbcIO?
> On Mon, Oct 15, 2018 at 2:47 PM Jean-Baptiste Onofré  
> wrote:
>>
>> If you want to reuse MongoDbIO, there's no easy way.
>>
>> However, I can introduce the same change we did in Jdbc or Elasticsearch
>> IOs.
>>
>> Agree ?
>>
>> Regards
>> JB
>>
>> On 15/10/2018 13:46, Chaim Turkel wrote:
>>> Thanks,
>>>   I need to wrap MongoDbIO.read, and don't see an easy way to do it
>>> chaim
>>> On Mon, Oct 15, 2018 at 2:30 PM Jean-Baptiste Onofré  
>>> wrote:

 Hi Chaim,

 you can take a look on JdbcIO.

 You can create any "startup" PCollection on the PBegin, and then you can
 can the DoFn based on that.

 Regards
 JB

 On 15/10/2018 13:00, Chaim Turkel wrote:
> Hi,
>   I there a way to write code before the PBegin.
> I am writeing a pipeline to connect to mongo with self signed ssl. I
> need to init the ssl connection of the java before the mongo code. So
> i need to write code before the PBegin but for it to run on each node?
>
>
> Chaim
>

 --
 Jean-Baptiste Onofré
 jbono...@apache.org
 http://blog.nanthrax.net
 Talend - http://www.talend.com
>>>
>>
>> --
>> Jean-Baptiste Onofré
>> jbono...@apache.org
>> http://blog.nanthrax.net
>> Talend - http://www.talend.com
> 

-- 
Jean-Baptiste Onofré
jbono...@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com


Re: PBegin

2018-10-15 Thread Chaim Turkel
what would be the implementation for the JdbcIO?
On Mon, Oct 15, 2018 at 2:47 PM Jean-Baptiste Onofré  wrote:
>
> If you want to reuse MongoDbIO, there's no easy way.
>
> However, I can introduce the same change we did in Jdbc or Elasticsearch
> IOs.
>
> Agree ?
>
> Regards
> JB
>
> On 15/10/2018 13:46, Chaim Turkel wrote:
> > Thanks,
> >   I need to wrap MongoDbIO.read, and don't see an easy way to do it
> > chaim
> > On Mon, Oct 15, 2018 at 2:30 PM Jean-Baptiste Onofré  
> > wrote:
> >>
> >> Hi Chaim,
> >>
> >> you can take a look on JdbcIO.
> >>
> >> You can create any "startup" PCollection on the PBegin, and then you can
> >> can the DoFn based on that.
> >>
> >> Regards
> >> JB
> >>
> >> On 15/10/2018 13:00, Chaim Turkel wrote:
> >>> Hi,
> >>>   I there a way to write code before the PBegin.
> >>> I am writeing a pipeline to connect to mongo with self signed ssl. I
> >>> need to init the ssl connection of the java before the mongo code. So
> >>> i need to write code before the PBegin but for it to run on each node?
> >>>
> >>>
> >>> Chaim
> >>>
> >>
> >> --
> >> Jean-Baptiste Onofré
> >> jbono...@apache.org
> >> http://blog.nanthrax.net
> >> Talend - http://www.talend.com
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com

-- 


Loans are funded by
FinWise Bank, a Utah-chartered bank located in Sandy, 
Utah, member FDIC, Equal
Opportunity Lender. Merchant Cash Advances are 
made by Behalf. For more
information on ECOA, click here 
. For important information about 
opening a new
account, review Patriot Act procedures here 
.
Visit Legal 
 to
review our comprehensive program terms, 
conditions, and disclosures. 


Re: [BEAM-5442] Store duplicate unknown (runner) options in a list argument

2018-10-15 Thread Maximilian Michels
I agree that the current approach breaks the pipeline options contract 
because "unknown" options get parsed in the same way as options which 
have been defined by the user.


I'm not sure the `experiments` flag works for us. AFAIK it only allows 
true/false flags. We want to pass all types of pipeline options to the 
Runner.


How to solve this?

1) Add all options of all Runners to each SDK
We added some of the FlinkRunner options to the Python SDK but realized 
syncing is rather cumbersome in the long term. However, we want the most 
important options to be validated on the client side.


2) Pass "unknown" options via a separate list in the Proto which can 
only be accessed internally by the Runners. This still allows passing 
arbitrary options but we wouldn't leak unknown options and display them 
as top-level options.


-Max

On 13.10.18 02:34, Charles Chen wrote:
The current release branch 
(https://github.com/apache/beam/commits/release-2.8.0) was cut after the 
revert went in.  Sent out https://github.com/apache/beam/pull/6683 as a 
revert of the revert.  Regarding your comment above, I can help out with 
the design / PR reviews for common Python code as you suggest.


On Fri, Oct 12, 2018 at 4:48 PM Thomas Weise > wrote:


Thanks, will tag you and looking forward to feedback so we can
ensure that changes work for everyone.

Looking at the PR, I see agreement from Max to revert the change on
the release branch, but not in master. Would you mind to restore it
in master?

Thanks

On Fri, Oct 12, 2018 at 4:40 PM Ahmet Altay mailto:al...@google.com>> wrote:



On Fri, Oct 12, 2018 at 11:31 AM, Charles Chen mailto:c...@google.com>> wrote:

What I mean is that a user may find that it works for them
to pass "--myarg blah" and access it as "options.myarg"
without explicitly defining a "my_arg" flag due to the added
logic.  This is not the intended behavior and we may want to
change this implementation detail in the future.  However,
having this logic in a released version makes it hard to
change this behavior since users may erroneously depend on
this undocumented behavior.  Instead, we should namespace /
scope this so that it is obvious that this is meant for
runner (and not Beam user) consumption.

On Fri, Oct 12, 2018 at 10:48 AM Thomas Weise
mailto:t...@apache.org>> wrote:

Can you please elaborate more what practical problems
this introduces for users?

I can see that this change allows a user to specify a
runner specific option, which in the future may change
because we decide to scope differently. If this only
affects users of the portable Flink runner (like us),
then no need to revert, because at this early stage we
prefer something that works over being blocked.

It would also be really great if some of the core Python
SDK developers could help out with the design aspects
and PR reviews of changes that affect common Python
code. Anyone who specifically wants to be tagged on
relevant JIRAs and PRs?


I would be happy to be tagged, and I can also help with
including other relevant folks whenever possible. In general I
think Robert, Charles, myself are good candidates.


Thanks


On Fri, Oct 12, 2018 at 10:20 AM Ahmet Altay
mailto:al...@google.com>> wrote:



On Fri, Oct 12, 2018 at 10:11 AM, Charles Chen
mailto:c...@google.com>> wrote:

For context, I made comments on
https://github.com/apache/beam/pull/6600 noting
that the changes being made were not good for
Beam backwards-compatibility.  The change as is
allows users to use pipeline options without
explicitly defining them, which is not the type
of usage we would like to encourage since we
prefer to be explicit whenever possible.  If
users write pipelines with this sort of pattern,
they will potentially encounter pain when
upgrading to a later version since this is an
implementation detail and not an officially
supported pattern.  I agree with the comments
above that this is ultimately a scoping issue. 
I would not have a problem with these changes if

they were explicitly scoped under either a
runner or unparsed options namespace.

Re: [DISCUSS] - Separate JIRA notifications to a new mailing list

2018-10-15 Thread Colm O hEigeartaigh
Excellent, thanks everyone! I am also +1 to the suggestion made in the
thread to separate out build mails into a separate mailing list as well.

Colm.

On Mon, Oct 15, 2018 at 2:18 PM Maximilian Michels  wrote:

> The list has been created. You can already subscribe to the list by
> sending an empty mail to: issues-subscr...@beam.apache.org
>
> Could we announce the time when we plan to make the switch to the new
> list? Also, the mailing list page needs to be updated:
> https://beam.apache.org/community/contact-us/
>
> On 11.10.18 23:21, Rui Wang wrote:
> > +1
> >
> > -Rui
> >
> > On Thu, Oct 11, 2018 at 12:53 PM Ankur Goenka  > > wrote:
> >
> > +1
> >
> > On Thu, Oct 11, 2018 at 12:14 PM Suneel Marthi
> > mailto:suneel.mar...@gmail.com>> wrote:
> >
> > +1
> >
> > Sent from my iPhone
> >
> > On Oct 11, 2018, at 8:03 PM, Łukasz Gajowy
> > mailto:lukasz.gaj...@gmail.com>>
> wrote:
> >
> >> This is a good idea. +1
> >>
> >> Łukasz
> >>
> >>
> >> czw., 11 paź 2018, 18:01 użytkownik Udi Meiri
> >> mailto:eh...@google.com>> napisał:
> >>
> >> +1 to split JIRA notifications
> >>
> >> On Thu, Oct 11, 2018 at 9:13 AM Kenneth Knowles
> >> mailto:k...@apache.org>> wrote:
> >>
> >>
> >> On Thu, Oct 11, 2018 at 9:10 AM Mikhail Gryzykhin
> >>  >> > wrote:
> >>
> >> +1.
> >> Should we separate Jenkins notifications as well?
> >>
> >>
> >> I'm worried this question will get buried in the
> >> thread. Would you mind separating it into another
> >> thread if you would like to discuss?
> >>
> >> Kenn
> >>
> >> On Thu, Oct 11, 2018, 08:59 Scott Wegner
> >> mailto:sc...@apache.org> wrote:
> >>
> >> +1, commits@ is too noisy to be useful
> currently.
> >>
> >> On Thu, Oct 11, 2018 at 8:04 AM Maximilian
> >> Michels  >> > wrote:
> >>
> >> +1
> >>
> >> I guess most people have already filters
> >> in place to separate commits
> >> and JIRA issues. JIRA really has nothing
> >> to do in the commits list.
> >>
> >> On 11.10.18 15:53, Kenneth Knowles wrote:
> >> > +1
> >> >
> >> > I've suggested the same. Canonical.
> >> >
> >> > On Thu, Oct 11, 2018, 06:19 Thomas Weise
> >> mailto:t...@apache.org>
> >> >  >> >> wrote:
> >> >
> >> > +1
> >> >
> >> >
> >> > On Thu, Oct 11, 2018 at 6:18 AM
> >> Etienne Chauchot
> >> >  >> 
> >>  >> >> wrote:
> >> >
> >> > +1 for me also, my gmail filters
> >> list is kind of overflowed :)
> >> >
> >> > Etienne
> >> >
> >> > Le jeudi 11 octobre 2018 à 14:44
> >> +0200, Robert Bradshaw a écrit :
> >> >> Huge +1 from me too.
> >> >> On Thu, Oct 11, 2018 at 2:42 PM
> >> Jean-Baptiste Onofré  >> 
> >>  >> >> wrote:
> >> >>
> >> >> +1
> >> >>
> >> >> We are doing the same in Karaf
> >> as well.
> >> >>
> >> >> Regards
> >> >> JB
> >> >>
> >> >> On 11/10/2018 14:35, Colm O
> >> hEigeartaigh wrote:
> >> >> Hi all,
> >> >>

Re: [DISCUSS] - Separate JIRA notifications to a new mailing list

2018-10-15 Thread Maximilian Michels
The list has been created. You can already subscribe to the list by 
sending an empty mail to: issues-subscr...@beam.apache.org


Could we announce the time when we plan to make the switch to the new 
list? Also, the mailing list page needs to be updated: 
https://beam.apache.org/community/contact-us/


On 11.10.18 23:21, Rui Wang wrote:

+1

-Rui

On Thu, Oct 11, 2018 at 12:53 PM Ankur Goenka > wrote:


+1

On Thu, Oct 11, 2018 at 12:14 PM Suneel Marthi
mailto:suneel.mar...@gmail.com>> wrote:

+1

Sent from my iPhone

On Oct 11, 2018, at 8:03 PM, Łukasz Gajowy
mailto:lukasz.gaj...@gmail.com>> wrote:


This is a good idea. +1

Łukasz


czw., 11 paź 2018, 18:01 użytkownik Udi Meiri
mailto:eh...@google.com>> napisał:

+1 to split JIRA notifications

On Thu, Oct 11, 2018 at 9:13 AM Kenneth Knowles
mailto:k...@apache.org>> wrote:


On Thu, Oct 11, 2018 at 9:10 AM Mikhail Gryzykhin
mailto:gryzykhin.mikh...@gmail.com>> wrote:

+1.
Should we separate Jenkins notifications as well?


I'm worried this question will get buried in the
thread. Would you mind separating it into another
thread if you would like to discuss?

Kenn

On Thu, Oct 11, 2018, 08:59 Scott Wegner
mailto:sc...@apache.org> wrote:

+1, commits@ is too noisy to be useful currently.

On Thu, Oct 11, 2018 at 8:04 AM Maximilian
Michels mailto:m...@apache.org>> wrote:

+1

I guess most people have already filters
in place to separate commits
and JIRA issues. JIRA really has nothing
to do in the commits list.

On 11.10.18 15:53, Kenneth Knowles wrote:
> +1
>
> I've suggested the same. Canonical.
>
> On Thu, Oct 11, 2018, 06:19 Thomas Weise
mailto:t...@apache.org>
> >> wrote:
>
>     +1
>
>
>     On Thu, Oct 11, 2018 at 6:18 AM
Etienne Chauchot
>     mailto:echauc...@apache.org>
>> wrote:
>
>         +1 for me also, my gmail filters
list is kind of overflowed :)
>
>         Etienne
>
>         Le jeudi 11 octobre 2018 à 14:44
+0200, Robert Bradshaw a écrit :
>>         Huge +1 from me too.
>>         On Thu, Oct 11, 2018 at 2:42 PM
Jean-Baptiste Onofré mailto:j...@nanthrax.net>
>> wrote:
>>
>>         +1
>>
>>         We are doing the same in Karaf
as well.
>>
>>         Regards
>>         JB
>>
>>         On 11/10/2018 14:35, Colm O
hEigeartaigh wrote:
>>         Hi all,
>>
>>         Apologies in advance if this
has already been discussed (and rejected).
>>         I was wondering if it would be
a good idea to create a new mailing list
>>         and divert the JIRA
notifications to it? Currently
>>         "comm...@beam.apache.org

> 


Beam Dependency Check Report (2018-10-15)

2018-10-15 Thread Apache Jenkins Server

High Priority Dependency Updates Of Beam Python SDK:


  Dependency Name
  Current Version
  Latest Version
  Release Date Of the Current Used Version
  Release Date Of The Latest Release
  JIRA Issue
  
google-cloud-bigquery
0.25.0
1.6.0
2017-06-26
2017-06-26BEAM-5537
google-cloud-core
0.25.0
0.28.1
2017-06-26
2017-06-26BEAM-5538
google-cloud-pubsub
0.26.0
0.38.0
2017-06-26
2017-06-26BEAM-5539
ply
3.8
3.11
2015-10-02
2015-10-02BEAM-5540
High Priority Dependency Updates Of Beam Java SDK:


  Dependency Name
  Current Version
  Latest Version
  Release Date Of the Current Used Version
  Release Date Of The Latest Release
  JIRA Issue
  
com.google.auto.service:auto-service
1.0-rc2
1.0-rc4
2014-10-25
2017-12-11BEAM-5541
com.gradle:build-scan-plugin
1.13.1
1.16
2018-04-10
2018-08-27BEAM-5543
org.conscrypt:conscrypt-openjdk
1.1.3
1.4.0
2018-06-04
2018-10-03BEAM-5748
org.elasticsearch:elasticsearch-hadoop
5.0.0
6.4.2
2016-10-26
2018-09-26BEAM-5551
com.google.code.gson:gson
2.7
2.8.5
2016-06-14
2018-05-22BEAM-5558
org.apache.hbase:hbase-common
1.2.6
2.1.0
2017-05-29
2018-07-10BEAM-5560
org.apache.hbase:hbase-hadoop-compat
1.2.6
2.1.0
2017-05-29
2018-07-10BEAM-5561
org.apache.hbase:hbase-hadoop2-compat
1.2.6
2.1.0
2017-05-29
2018-07-10BEAM-5562
org.apache.hbase:hbase-server
1.2.6
2.1.0
2017-05-29
2018-07-10BEAM-5563
org.apache.hbase:hbase-shaded-client
1.2.6
2.1.0
2017-05-29
2018-07-10BEAM-5564
org.apache.hive:hive-cli
2.1.0
3.1.0
2016-06-17
2018-07-23BEAM-5566
org.apache.hive:hive-common
2.1.0
3.1.0
2016-06-17
2018-07-23BEAM-5567
org.apache.hive:hive-exec
2.1.0
3.1.0
2016-06-17
2018-07-23BEAM-5568
org.apache.hive.hcatalog:hive-hcatalog-core
2.1.0
3.1.0
2016-06-17
2018-07-23BEAM-5569
net.java.dev.javacc:javacc
4.0
7.0.4
2006-03-17
2018-09-17BEAM-5570
javax.servlet:javax.servlet-api
3.1.0
4.0.1
2013-04-25
2018-04-20BEAM-5750
org.eclipse.jetty:jetty-server
9.2.10.v20150310
9.4.12.RC2
2015-03-10
2018-08-14BEAM-5752
org.eclipse.jetty:jetty-servlet
9.2.10.v20150310
9.4.12.RC2
2015-03-10
2018-08-14BEAM-5753
net.java.dev.jna:jna
4.1.0
5.0.0
2014-03-06
2018-10-07BEAM-5573
com.esotericsoftware.kryo:kryo
2.21
2.24.0
2013-02-27
2014-05-04BEAM-5574
org.apache.kudu:kudu-client
1.4.0
1.7.1
2017-06-05
2018-05-30BEAM-5575
io.dropwizard.metrics:metrics-core
3.1.2
4.1.0-rc2
2015-04-26
2018-05-03BEAM-5576
org.mongodb:mongo-java-driver
3.2.2
3.8.2
2016-02-15
2018-09-19BEAM-5577
org.apache.qpid:proton-j
0.13.1
0.29.0
2016-07-02
2018-08-10BEAM-5582
org.apache.solr:solr-core
5.5.4
7.5.0
2017-02-14
2018-09-22BEAM-5589
org.apache.solr:solr-solrj
5.5.4
7.5.0
2017-02-14
2018-09-22BEAM-5590
org.apache.solr:solr-test-framework
5.5.4
7.5.0
2017-02-14
2018-09-22BEAM-5591
com.diffplug.spotless:spotless-plugin-gradle
3.6.0
3.15.0
2017-09-29
2018-09-24BEAM-5592
org.elasticsearch.client:transport
5.0.0
6.4.2
2016-10-26
2018-09-26BEAM-5595

 A dependency update is high priority if it satisfies one of following criteria: 

 It has major versions update available, e.g. org.assertj:assertj-core 2.5.0 -> 3.10.0; 


 It is over 3 minor versions behind the latest version, e.g. org.tukaani:xz 1.5 -> 1.8; 


 The current version is behind the later version for over 180 days, e.g. com.google.auto.service:auto-service 2014-10-24 -> 2017-12-11. 

 In Beam, we make a best-effort attempt at keeping all dependencies up-to-date.
 In the future, issues will be filed and tracked for these automatically,
 but in the meantime you can search for existing issues or open a new one.

 For more information:  Beam Dependency Guide  

Re: PBegin

2018-10-15 Thread Jean-Baptiste Onofré
If you want to reuse MongoDbIO, there's no easy way.

However, I can introduce the same change we did in Jdbc or Elasticsearch
IOs.

Agree ?

Regards
JB

On 15/10/2018 13:46, Chaim Turkel wrote:
> Thanks,
>   I need to wrap MongoDbIO.read, and don't see an easy way to do it
> chaim
> On Mon, Oct 15, 2018 at 2:30 PM Jean-Baptiste Onofré  
> wrote:
>>
>> Hi Chaim,
>>
>> you can take a look on JdbcIO.
>>
>> You can create any "startup" PCollection on the PBegin, and then you can
>> can the DoFn based on that.
>>
>> Regards
>> JB
>>
>> On 15/10/2018 13:00, Chaim Turkel wrote:
>>> Hi,
>>>   I there a way to write code before the PBegin.
>>> I am writeing a pipeline to connect to mongo with self signed ssl. I
>>> need to init the ssl connection of the java before the mongo code. So
>>> i need to write code before the PBegin but for it to run on each node?
>>>
>>>
>>> Chaim
>>>
>>
>> --
>> Jean-Baptiste Onofré
>> jbono...@apache.org
>> http://blog.nanthrax.net
>> Talend - http://www.talend.com
> 

-- 
Jean-Baptiste Onofré
jbono...@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com


Re: PBegin

2018-10-15 Thread Chaim Turkel
Thanks,
  I need to wrap MongoDbIO.read, and don't see an easy way to do it
chaim
On Mon, Oct 15, 2018 at 2:30 PM Jean-Baptiste Onofré  wrote:
>
> Hi Chaim,
>
> you can take a look on JdbcIO.
>
> You can create any "startup" PCollection on the PBegin, and then you can
> can the DoFn based on that.
>
> Regards
> JB
>
> On 15/10/2018 13:00, Chaim Turkel wrote:
> > Hi,
> >   I there a way to write code before the PBegin.
> > I am writeing a pipeline to connect to mongo with self signed ssl. I
> > need to init the ssl connection of the java before the mongo code. So
> > i need to write code before the PBegin but for it to run on each node?
> >
> >
> > Chaim
> >
>
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com

-- 


Loans are funded by
FinWise Bank, a Utah-chartered bank located in Sandy, 
Utah, member FDIC, Equal
Opportunity Lender. Merchant Cash Advances are 
made by Behalf. For more
information on ECOA, click here 
. For important information about 
opening a new
account, review Patriot Act procedures here 
.
Visit Legal 
 to
review our comprehensive program terms, 
conditions, and disclosures. 


Re: PBegin

2018-10-15 Thread Jean-Baptiste Onofré
Hi Chaim,

you can take a look on JdbcIO.

You can create any "startup" PCollection on the PBegin, and then you can
can the DoFn based on that.

Regards
JB

On 15/10/2018 13:00, Chaim Turkel wrote:
> Hi,
>   I there a way to write code before the PBegin.
> I am writeing a pipeline to connect to mongo with self signed ssl. I
> need to init the ssl connection of the java before the mongo code. So
> i need to write code before the PBegin but for it to run on each node?
> 
> 
> Chaim
> 

-- 
Jean-Baptiste Onofré
jbono...@apache.org
http://blog.nanthrax.net
Talend - http://www.talend.com


PBegin

2018-10-15 Thread Chaim Turkel
Hi,
  I there a way to write code before the PBegin.
I am writeing a pipeline to connect to mongo with self signed ssl. I
need to init the ssl connection of the java before the mongo code. So
i need to write code before the PBegin but for it to run on each node?


Chaim

-- 


Loans are funded by
FinWise Bank, a Utah-chartered bank located in Sandy, 
Utah, member FDIC, Equal
Opportunity Lender. Merchant Cash Advances are 
made by Behalf. For more
information on ECOA, click here 
. For important information about 
opening a new
account, review Patriot Act procedures here 
.
Visit Legal 
 to
review our comprehensive program terms, 
conditions, and disclosures. 


Re: [FYI] Paper of Building Beam Runner for IBM Streams

2018-10-15 Thread Alexey Romanenko
Really great paper, thank you for sharing!

> On 10 Sep 2018, at 18:38, Tim  wrote:
> 
> Thanks for sharing Manu - interesting paper indeed.
> 
> Tim
> 
>> On 10 Sep 2018, at 16:02, Maximilian Michels  wrote:
>> 
>> Excellent write-up. Thank you!
>> 
>>> On 09.09.18 20:43, Jean-Baptiste Onofré wrote:
>>> Good idea. It could also help people who wants to create runners.
>>> Regards
>>> JB
 On 09/09/2018 13:00, Manu Zhang wrote:
 Hi all,
 
 I've spent the weekend reading Challenges and Experiences in Building an
 Efficient Apache Beam Runner For IBM Streams
  from the August
 proceeding of PVLDB. It's quite enjoyable and urges me to reflect on how
 I (should've) implemented the Gearpump runner. I believe it will be
 beneficial to have more such papers and discussions as sharing design
 choices and lessons from various runners.
 
 Enjoy it !
 Manu Zhang