Re: Question on SerializableCoder

2020-03-27 Thread Colm O hEigeartaigh
Thanks Luke. I updated the JIRA to suggest changing the docs along the
lines of your post: https://issues.apache.org/jira/browse/BEAM-9570

Colm.

On Thu, Mar 26, 2020 at 3:06 PM Luke Cwik  wrote:

> From the private@ thread, I suggested:
> "With the JvmInitializer[1] being supported by Dataflow and the portable
> Java container, users would be able to write code which sets the system
> property jdk.serialFilter or by configuring
> ObjectInputFilter.Config.setSerialFilter(filter)[2]"
>
> This could become a documentation change to SerializableCoder.
>
> 1:
> https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/harness/JvmInitializer.java
> 2:
> https://docs.oracle.com/javase/10/core/serialization-filtering1.htm#JSCOR-GUID-952E2328-AB66-4412-8B6B-3BCCB3195C25
>
> On Thu, Mar 26, 2020 at 4:51 AM Colm O hEigeartaigh 
> wrote:
>
>> Thanks for all the feedback. Two possible ideas that occur to me:
>>
>>  - Create TypeSafeSerializableCoder or somesuch which extends
>> SerializableCoder and enforces the check as in the PR. Update the
>> documentation to suggest using the new coder if you don't need to support
>> collections as the raw type or subclasses.
>>  - Add a new "of" method to SerializableCoder which takes a boolean
>> parameter to control whether we perform this check or not (defaults to
>> true?).
>>
>> Colm.
>>
>> On Tue, Mar 24, 2020 at 2:11 AM Luke Cwik  wrote:
>>
>>> I have seen people ingest data using SerializableCoder.
>>>
>>> On Mon, Mar 23, 2020 at 2:51 PM Kenneth Knowles  wrote:
>>>
>>>> I won't bring other people's words from private@, but can share mine.
>>>> I don't believe it exposes anything new.
>>>>
>>>> > If it is SerializableCoder - attacker controls the other end of e.g.
>>>> Kafka or Pubsub that is decoding w/ ObjectInputStream - [then we could have
>>>> an allowlist or try to automatically construct an allowlist] and otherwise
>>>> there is no vulnerability for internal coders.
>>>>
>>>> I have never seen or heard of a user doing dynamic deserialization
>>>> dispatch on ingestion, but that doesn't mean it doesn't happen. If it is
>>>> important to someone then they would need a more secure solution than
>>>> SerializableCoder.
>>>>
>>>> Side note: it would be great to provide an efficient and usable
>>>> solution for the problem of wanting to dynamically dispatch serde in the
>>>> middle of a pipeline. It is actually independent from being able to provide
>>>> coders for a wide variety of types, which we can do a bunch of different
>>>> (mostly better) ways. (has a better solution been built since the last time
>>>> I thought about this?)
>>>>
>>>> Kenn
>>>>
>>>> On Mon, Mar 23, 2020 at 1:36 PM Ismaël Mejía  wrote:
>>>>
>>>>> The link to the previous covnersation (discussion happened in private@
>>>>> and I suppose we can bring some relevant bits here if needed)
>>>>>
>>>>> https://lists.apache.org/thread.html/2e1c00999e992e15b08938866bfe7bd3c3d3b3d4d7aa2f8f6eb4600d%40%3Cprivate.beam.apache.org%3E
>>>>>
>>>>> I remember Robert had some points there, but I am not sure we
>>>>> found/agreed on a solution that was relevant and did not break current
>>>>> users and their user experience (like the case of blacklists).
>>>>>
>>>>> On Mon, Mar 23, 2020 at 9:22 PM Luke Cwik  wrote:
>>>>> >
>>>>> > Being able to have something that can encode any object (or at least
>>>>> a large class of objects) is extremely powerful so requiring
>>>>> SerializableCoder to only encode T.class would hurt our users.
>>>>> >
>>>>> > I believe someone looked at this kind of problem before and we came
>>>>> to an agreement of usng an explicit approve/deny list on the class names
>>>>> which would address the security concern. I don't remember the thread
>>>>> though and couldn't find the thread after a few minutes of searching.
>>>>> >
>>>>> >
>>>>> >
>>>>> > On Mon, Mar 23, 2020 at 1:07 PM Kenneth Knowles 
>>>>> wrote:
>>>>> >>
>>>>> >> So you think the spec for SerializableCoder (currently doesn't
>>>>> really have one) should be that it dynamically dispatches what

Re: Question on SerializableCoder

2020-03-26 Thread Colm O hEigeartaigh
Thanks for all the feedback. Two possible ideas that occur to me:

 - Create TypeSafeSerializableCoder or somesuch which extends
SerializableCoder and enforces the check as in the PR. Update the
documentation to suggest using the new coder if you don't need to support
collections as the raw type or subclasses.
 - Add a new "of" method to SerializableCoder which takes a boolean
parameter to control whether we perform this check or not (defaults to
true?).

Colm.

On Tue, Mar 24, 2020 at 2:11 AM Luke Cwik  wrote:

> I have seen people ingest data using SerializableCoder.
>
> On Mon, Mar 23, 2020 at 2:51 PM Kenneth Knowles  wrote:
>
>> I won't bring other people's words from private@, but can share mine. I
>> don't believe it exposes anything new.
>>
>> > If it is SerializableCoder - attacker controls the other end of e.g.
>> Kafka or Pubsub that is decoding w/ ObjectInputStream - [then we could have
>> an allowlist or try to automatically construct an allowlist] and otherwise
>> there is no vulnerability for internal coders.
>>
>> I have never seen or heard of a user doing dynamic deserialization
>> dispatch on ingestion, but that doesn't mean it doesn't happen. If it is
>> important to someone then they would need a more secure solution than
>> SerializableCoder.
>>
>> Side note: it would be great to provide an efficient and usable solution
>> for the problem of wanting to dynamically dispatch serde in the middle of a
>> pipeline. It is actually independent from being able to provide coders for
>> a wide variety of types, which we can do a bunch of different (mostly
>> better) ways. (has a better solution been built since the last time I
>> thought about this?)
>>
>> Kenn
>>
>> On Mon, Mar 23, 2020 at 1:36 PM Ismaël Mejía  wrote:
>>
>>> The link to the previous covnersation (discussion happened in private@
>>> and I suppose we can bring some relevant bits here if needed)
>>>
>>> https://lists.apache.org/thread.html/2e1c00999e992e15b08938866bfe7bd3c3d3b3d4d7aa2f8f6eb4600d%40%3Cprivate.beam.apache.org%3E
>>>
>>> I remember Robert had some points there, but I am not sure we
>>> found/agreed on a solution that was relevant and did not break current
>>> users and their user experience (like the case of blacklists).
>>>
>>> On Mon, Mar 23, 2020 at 9:22 PM Luke Cwik  wrote:
>>> >
>>> > Being able to have something that can encode any object (or at least a
>>> large class of objects) is extremely powerful so requiring
>>> SerializableCoder to only encode T.class would hurt our users.
>>> >
>>> > I believe someone looked at this kind of problem before and we came to
>>> an agreement of usng an explicit approve/deny list on the class names which
>>> would address the security concern. I don't remember the thread though and
>>> couldn't find the thread after a few minutes of searching.
>>> >
>>> >
>>> >
>>> > On Mon, Mar 23, 2020 at 1:07 PM Kenneth Knowles 
>>> wrote:
>>> >>
>>> >> So you think the spec for SerializableCoder (currently doesn't
>>> really have one) should be that it dynamically dispatches what it
>>> deserializes? I had imagined we would treat it more as a statically
>>> determined coder, so because it is invariant in T we would not allow up or
>>> down casts (they are unsafe). But we probably don't actually have the
>>> static information to do that anyhow so you are probably right.
>>> >>
>>> >> I wonder about the threat model here. Is this the event that the
>>> runner (managed service or bespoke cluster) is compromised and is
>>> attempting RCE on the Java SDK harness or runner-specific Java-based worker?
>>> >>
>>> >> Kenn
>>> >>
>>> >> On Mon, Mar 23, 2020 at 8:09 AM Luke Cwik  wrote:
>>> >>>
>>> >>> I don't think this is going to work since SerializableCoder
>>> should be able to decode T and all objects that implement/extend T. I'm
>>> pretty sure SerializableCoder is common enough while the
>>> concrete type is HashSet/ArrayList/...
>>> >>> I'm pretty sure there is some way you could come up with some way
>>> for making this optin though.
>>> >>>
>>> >>> On Mon, Mar 23, 2020 at 12:19 AM Colm O hEigeartaigh <
>>> cohei...@apache.org> wrote:
>>> >>>>
>>> >>>> Thanks Kenn. I submitted a PR here:
>>> https://github.com/apache/b

Re: Question on SerializableCoder

2020-03-23 Thread Colm O hEigeartaigh
Thanks Kenn. I submitted a PR here:
https://github.com/apache/beam/pull/11191

Colm.

On Thu, Mar 19, 2020 at 8:13 PM Kenneth Knowles  wrote:

> I think this is fine. The same coder is used for encode and decode, so the
> Class object should be the same as well. Inheritance is not part of the
> Beam model (thank goodness) so this is a language-specific concern. As far
> as the model is concerned, the full URN and the payload of the coder is its
> identity and coders with different identities have no inheritance or
> compatibility relationship. Pipeline snapshot/update is an edge case, but
> changing coder is not supported by any runner I know of, and probably won't
> be until we have some rather large new ideas.
>
> Kenn
>
> On Thu, Mar 19, 2020 at 4:50 AM Colm O hEigeartaigh 
> wrote:
>
>> Hi,
>>
>> I have a question on SerializableCoder. I'm looking at hardening the Java
>> Object deserialization that is taking place. We have a "Class type" that
>> is used to decode the input stream:
>>
>> ObjectInputStream ois = new ObjectInputStream(inStream);
>> return type.cast(ois.readObject());
>>
>> What I would like to do would be something like:
>>
>> ObjectInputStream ois = new ObjectInputStream(inStream) {
>> @Override
>> protected Class resolveClass(ObjectStreamClass desc) throws
>> IOException, ClassNotFoundException {
>> if (!desc.getName().equals(type.getName())) {
>> throw new InvalidClassException("Unauthorized deserialization
>> attempt", desc.getName());
>> }
>> return super.resolveClass(desc);
>> }
>> };
>> return type.cast(ois.readObject());
>>
>> This would prevent a possible security hole where an attacker could try
>> to force the recipient of the input stream to deserialize to a gadget class
>> or the like for a RCE.
>>
>> The question is - does the deserialized type have to correspond exactly
>> to the supplied Class? Or is it supported that it's a base type / abstract
>> class? If the latter then my idea won't really work. But if the type
>> corresponds exactly then it should work OK.
>>
>> Thanks,
>>
>> Colm.
>>
>


Question on SerializableCoder

2020-03-19 Thread Colm O hEigeartaigh
Hi,

I have a question on SerializableCoder. I'm looking at hardening the Java
Object deserialization that is taking place. We have a "Class type" that
is used to decode the input stream:

ObjectInputStream ois = new ObjectInputStream(inStream);
return type.cast(ois.readObject());

What I would like to do would be something like:

ObjectInputStream ois = new ObjectInputStream(inStream) {
@Override
protected Class resolveClass(ObjectStreamClass desc) throws
IOException, ClassNotFoundException {
if (!desc.getName().equals(type.getName())) {
throw new InvalidClassException("Unauthorized deserialization
attempt", desc.getName());
}
return super.resolveClass(desc);
}
};
return type.cast(ois.readObject());

This would prevent a possible security hole where an attacker could try to
force the recipient of the input stream to deserialize to a gadget class or
the like for a RCE.

The question is - does the deserialized type have to correspond exactly to
the supplied Class? Or is it supported that it's a base type / abstract
class? If the latter then my idea won't really work. But if the type
corresponds exactly then it should work OK.

Thanks,

Colm.


Re: [RELEASE] Tracking 2.18

2019-12-05 Thread Colm O hEigeartaigh
Could we get this one in 2.18 as well?
https://issues.apache.org/jira/browse/BEAM-8861

Colm.

On Wed, Dec 4, 2019 at 8:02 PM Udi Meiri  wrote:

> Following the release calendar, I plan on cutting the 2.18 release branch
> today.
>
> There are currently 8 release blockers
> .
>
>


Re: Python docs build error

2018-10-22 Thread Colm O hEigeartaigh
Great, thanks! Out of curiosity, did the jenkins job for the initial PR not
detect the build failure?

Colm.

On Mon, Oct 22, 2018 at 2:29 PM Maximilian Michels  wrote:

> Correction for the footnote:
>
> [1] https://github.com/apache/beam/pull/6637
>
> On 22.10.18 15:24, Maximilian Michels wrote:
> > Hi Colm,
> >
> > This [1] got merged recently and broke the "docs" target which
> > apparently is not part of our Python PreCommit tests.
> >
> > See the following PR for a fix: https://github.com/apache/beam/pull/6774
> >
> > Best,
> > Max
> >
> > [1] https://github.com/apache/beam/pull/6737
> >
> > On 22.10.18 12:55, Colm O hEigeartaigh wrote:
> >> Hi all,
> >>
> >> The following command: ./gradlew :beam-sdks-python:docs gives me the
> >> following error:
> >>
> >>
> /home/coheig/src/apache/beam/sdks/python/apache_beam/io/flink/flink_streaming_impulse_source.py:docstring
>
> >> of
> >>
> apache_beam.io.flink.flink_streaming_impulse_source.FlinkStreamingImpulseSource.from_runner_api_parameter:11:
>
> >> WARNING: Unexpected indentation.
> >> Command exited with non-zero status 1
> >> 42.81user 4.02system 0:16.27elapsed 287%CPU (0avgtext+0avgdata
> >> 141036maxresident)k
> >> 0inputs+47792outputs (0major+727274minor)pagefaults 0swaps
> >> ERROR: InvocationError for command '/usr/bin/time
> >> /home/coheig/src/apache/beam/sdks/python/scripts/generate_pydoc.sh'
> >> (exited with code 1)
> >> ___ summary
> >> ____
> >> ERROR:   docs: commands failed
> >>
> >>  > Task :beam-sdks-python:docs FAILED
> >>
> >> FAILURE: Build failed with an exception.
> >>
> >> Am I missing something or is there an issue here?
> >>
> >> Thanks,
> >>
> >> Colm.
> >>
> >>
> >> --
> >> Colm O hEigeartaigh
> >>
> >> Talend Community Coder
> >> http://coders.talend.com
>


-- 
Colm O hEigeartaigh

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


Python docs build error

2018-10-22 Thread Colm O hEigeartaigh
Hi all,

The following command: ./gradlew :beam-sdks-python:docs gives me the
following error:

/home/coheig/src/apache/beam/sdks/python/apache_beam/io/flink/flink_streaming_impulse_source.py:docstring
of
apache_beam.io.flink.flink_streaming_impulse_source.FlinkStreamingImpulseSource.from_runner_api_parameter:11:
WARNING: Unexpected indentation.
Command exited with non-zero status 1
42.81user 4.02system 0:16.27elapsed 287%CPU (0avgtext+0avgdata
141036maxresident)k
0inputs+47792outputs (0major+727274minor)pagefaults 0swaps
ERROR: InvocationError for command '/usr/bin/time
/home/coheig/src/apache/beam/sdks/python/scripts/generate_pydoc.sh' (exited
with code 1)
___ summary

ERROR:   docs: commands failed

> Task :beam-sdks-python:docs FAILED

FAILURE: Build failed with an exception.

Am I missing something or is there an issue here?

Thanks,

Colm.


-- 
Colm O hEigeartaigh

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


Re: [DISCUSS] Move beam_SeedJob notifications to another email address

2018-10-22 Thread Colm O hEigeartaigh
We had a discussion recently about splitting the JIRA notifications to a
new list "issues@b.a.o", and also splitting the Jenkins mails to
(potentially) "builds@b.a.o". So I guess the issue you raised can be done
once the latter mailing list is set up and active.

Colm.

On Fri, Oct 19, 2018 at 11:29 PM Rui Wang  wrote:

> Hi Community,
>
> I have seen some Jenkins build failure/back-to-normal emails in dev@ in
> last several months. Seems to me that this setting is coded in
> https://github.com/apache/beam/blob/master/.test-infra/jenkins/job_00_seed.groovy#L100
> .
>
> In the link above, the comment says the seed job is very important so the
> notification emails should be sent to dev@.
>
> I am wondering if this is still true that we always want to see such
> notifications in dev@? If such notifications have become spams to dev@,
> can we move it to either commits@ or another dedicated email address
> (maybe create a new one)?
>
> -Rui
>


-- 
Colm O hEigeartaigh

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


Re: python tests run as part of 'javaPreCommit'

2018-10-16 Thread Colm O hEigeartaigh
Thanks Kenn, rookie mistake on my part :-)

A further question if I may - "./gradlew :javaPreCommit" is failing for me
with:

org.apache.beam.examples.WindowedWordCountIT >
testWindowedWordCountInBatchDynamicSharding FAILED
org.apache.beam.sdk.Pipeline$PipelineExecutionException at
WindowedWordCountIT.java:188

4 tests completed, 4 failed

> Task :beam-examples-java:directRunnerPreCommit FAILED

Looking at the report I see:

"Caused by: java.lang.RuntimeException: Unable to get application default
credentials. Please see
https://developers.google.com/accounts/docs/application-default-credentials
for details on how to specify credentials. This version of the SDK is
dependent on the gcloud core component version 2015.02.05 or newer to be
able to get credentials from the currently authorized user via gcloud auth."

It looks like some of the examples require google credentials to run
properly?

Colm.

On Tue, Oct 16, 2018 at 4:07 PM Kenneth Knowles  wrote:

> One thing to clarify is that `:javaPreCommit` is a task and `build` is
> another task. There's so verb-object relationship in your commandline. So
> as written, you've asked for a whole-project `build`, which weirdly in
> Gradle means "build and test". Since it is one commandline, all the
> necessary steps for both tasks will be in one dependency graph so they
> won't be executed twice.
>
> Kenn
>
> On Tue, Oct 16, 2018 at 6:11 AM Colm O hEigeartaigh 
> wrote:
>
>> Hi all,
>>
>> Just a quick question - I was wondering why the python tests/build run as
>> part of the 'javaPreCommit' task?
>>
>> i.e. executing "./gradlew build :javaPreCommit" leads to python tests
>> being run as well, which is not something you might expect from the name of
>> the task.
>>
>> Colm.
>>
>>
>> --
>> Colm O hEigeartaigh
>>
>> Talend Community Coder
>> http://coders.talend.com
>>
>

-- 
Colm O hEigeartaigh

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


python tests run as part of 'javaPreCommit'

2018-10-16 Thread Colm O hEigeartaigh
Hi all,

Just a quick question - I was wondering why the python tests/build run as
part of the 'javaPreCommit' task?

i.e. executing "./gradlew build :javaPreCommit" leads to python tests being
run as well, which is not something you might expect from the name of the
task.

Colm.


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


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  > <mailto:goe...@google.com>> 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>
> >> > <mailto:t...@apache.org
> >> <mailto:t...@apache.org>>> wrote:
> >> >
> >> > +1
> >> >
> >> >
> >> > On Thu, Oct 11, 2018 at 6:18 AM
> >> Etienne Chauchot
> >> >  >> <mailto:echauc...@apache.org>
> >> <mailto:echauc...@apache.org
> >> <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 :
> >> >

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

2018-10-11 Thread Colm O hEigeartaigh
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"
receives both the git and JIRA notifications, and has a huge volume of
traffic as a result.

Separating JIRA notifications from commit messages would allow users to
subscribe to whichever are of interest without having to write a mail
filter if e.g. they are not interested in JIRA notifications. It also seems
a bit unintuitive to me to expect JIRA notifications to go to an email list
called "commits".

As a reference point - Apache CXF maintains a "commits" list for git
notifications and "issues" for JIRA notifications:

http://cxf.apache.org/mailing-lists.html

Thanks!

Colm.

-- 
Colm O hEigeartaigh

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


Re: Go build failure

2018-05-29 Thread Colm O hEigeartaigh
Thanks guys, I've created the following JIRA:

https://issues.apache.org/jira/browse/BEAM-4422

Colm.

On Mon, May 28, 2018 at 5:36 PM, Jean-Baptiste Onofré 
wrote:

> ./gradlew build works but not check due to go:vet task.
>
> Let's check with Henning.
>
> Regards
> JB
>
> On 28/05/2018 16:57, Jean-Baptiste Onofré wrote:
> > Hi Colm,
> >
> > it worked for me (with build) during the weekend.
> >
> > Do you have golang installed on your machine ?
> >
> > Regards
> > JB
> >
> > On 28/05/2018 16:50, Colm O hEigeartaigh wrote:
> >> Hi all,
> >>
> >> The following fails for me with "./gradlew check":
> >>
> >> Execution failed for task ':beam-sdks-go:vet'.
> >>> Build failed due to return code 1 of:
> >>   Command:
> >>/home/coheig/.gradle/go/binary/1.10/go/bin/go tool vet
> >> /home/coheig/src/apache/beam/sdks/go/test
> >> /home/coheig/src/apache/beam/sdks/go/pkg
> >> /home/coheig/src/apache/beam/sdks/go/data
> >> /home/coheig/src/apache/beam/sdks/go/cmd
> >> /home/coheig/src/apache/beam/sdks/go/build
> >> /home/coheig/src/apache/beam/sdks/go/examples
> >> /home/coheig/src/apache/beam/sdks/go/container
> >>   Env:
> >>GOEXE=
> >>GOPATH=/home/coheig/src/apache/beam/sdks/go/.gogradle/project_gopath
> >>GOROOT=/home/coheig/.gradle/go/binary/1.10/go
> >>GOOS=linux
> >>GOARCH=amd64
> >>
> >> Should the gradle configuration be setting up Go, or is it a requirement
> >> to install it separately? Or is this failure caused by something else?
> >>
> >> Thanks,
> >>
> >> Colm.
> >>
> >>
> >> --
> >> Colm O hEigeartaigh
> >>
> >> Talend Community Coder
> >> http://coders.talend.com
> >
>
> --
> --
> Jean-Baptiste Onofré
> jbono...@apache.org
> http://blog.nanthrax.net
> Talend - http://www.talend.com
>



-- 
Colm O hEigeartaigh

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


Go build failure

2018-05-28 Thread Colm O hEigeartaigh
Hi all,

The following fails for me with "./gradlew check":

Execution failed for task ':beam-sdks-go:vet'.
> Build failed due to return code 1 of:
  Command:
   /home/coheig/.gradle/go/binary/1.10/go/bin/go tool vet
/home/coheig/src/apache/beam/sdks/go/test
/home/coheig/src/apache/beam/sdks/go/pkg
/home/coheig/src/apache/beam/sdks/go/data
/home/coheig/src/apache/beam/sdks/go/cmd
/home/coheig/src/apache/beam/sdks/go/build
/home/coheig/src/apache/beam/sdks/go/examples
/home/coheig/src/apache/beam/sdks/go/container
  Env:
   GOEXE=
   GOPATH=/home/coheig/src/apache/beam/sdks/go/.gogradle/project_gopath
   GOROOT=/home/coheig/.gradle/go/binary/1.10/go
   GOOS=linux
   GOARCH=amd64

Should the gradle configuration be setting up Go, or is it a requirement to
install it separately? Or is this failure caused by something else?

Thanks,

Colm.


-- 
Colm O hEigeartaigh

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


Maven build broken

2018-04-26 Thread Colm O hEigeartaigh
Hi,

This commit breaks the maven build:

commit 5d5c052e890fb8af4bb9cd1a152300542feba0a2
Author: Ben Sidhom <sid...@google.com>
Date:   Mon Apr 16 17:03:21 2018 -0700

[BEAM-4071] Add Portable Runner Job API shim

The changes to runners/reference/java/build.gradle were not also made to
the pom:

+  shadow project(path: ":beam-sdks-java-fn-execution", configuration:
"shadow")
   shadow library.java.slf4j_api
+  shadowTest project(path: ":beam-runners-core-construction-java",
configuration: "shadowTest")

Colm.


-- 
Colm O hEigeartaigh

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


Re: Go SDK build failures with maven

2018-04-20 Thread Colm O hEigeartaigh
Thanks for the quick turnaround! Both builds now working correctly for me.

Colm.

On Fri, Apr 20, 2018 at 12:50 AM, Henning Rohde <hero...@google.com> wrote:

> Hi Colm,
>
>   The extra pubsub dependency broke the Maven build. The warning you see
> in gradle seems to be a (non-breaking) linter check that the format string,
> which is interesting given that the function called is an internal one --
> but the check happens to be correct.
>
>  Sent you https://github.com/apache/beam/pull/5190 for both issues. Sorry
> about the noise.
>
> Thanks,
>  Henning
>
> On Thu, Apr 19, 2018 at 10:01 AM Henning Rohde <hero...@google.com> wrote:
>
>> The Go build works best with Gradle and that's what you should use (other
>> than possibly manually running go build etc). It looks like the build might
>> be broken due to the Go streaming PR independently of the build tooling.
>> Let me take a look.
>>
>> On Thu, Apr 19, 2018 at 7:09 AM Colm O hEigeartaigh <cohei...@apache.org>
>> wrote:
>>
>>> Hi all,
>>>
>>> Is the Apache Maven build still maintained? I'm seeing some recent
>>> failures in the Go SDK:
>>>
>>> [INFO] --- mvn-golang-wrapper:2.1.7:test (go-test) @ beam-sdks-go ---
>>> [INFO] Prepared command line : bin/go test ./...
>>> [ERROR]
>>> [ERROR] -Exec.Err-
>>> [ERROR] /home/colm/.mvnGoLang/.go_path/src/cloud.google.com/go/
>>> pubsub/subscription.go:30:2: cannot find package "
>>> golang.org/x/sync/errgroup" in any of:
>>> [ERROR] /home/colm/.mvnGoLang/go1.9.linux-amd64/src/golang.org/x/
>>> sync/errgroup (from $GOROOT)
>>> [ERROR] /home/colm/.mvnGoLang/.go_path/src/golang.org/x/sync/
>>> errgroup (from $GOPATH)
>>> [ERROR] /home/colm/src/apache/beam/sdks/go/target/src/golang.org/
>>> x/sync/errgroup
>>> [ERROR] /home/colm/.mvnGoLang/.go_path/src/cloud.google.com/go/
>>> pubsub/flow_controller.go:19:2: cannot find package "
>>> golang.org/x/sync/semaphore" in any of:
>>> [ERROR] /home/colm/.mvnGoLang/go1.9.linux-amd64/src/golang.org/x/
>>> sync/semaphore (from $GOROOT)
>>> [ERROR] /home/colm/.mvnGoLang/.go_path/src/golang.org/x/sync/
>>> semaphore (from $GOPATH)
>>> [ERROR] /home/colm/src/apache/beam/sdks/go/target/src/golang.org/
>>> x/sync/semaphore
>>> [ERROR]
>>>
>>> Incidentally, when run via gradle I see:
>>>
>>> /home/colm/src/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:75:
>>> wrong number of args for format in Errorf call: 1 needed but 2 args
>>> /home/colm/src/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:78:
>>> wrong number of args for format in Errorf call: 1 needed but 2 args
>>> /home/colm/src/apache/beam/sdks/go/target/src/github.com/
>>> apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:75: wrong number of
>>> args for format in Errorf call: 1 needed but 2 args
>>> /home/colm/src/apache/beam/sdks/go/target/src/github.com/
>>> apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:78: wrong number of
>>> args for format in Errorf call: 1 needed but 2 args
>>>
>>>
>>> Colm.
>>>
>>>
>>> --
>>> Colm O hEigeartaigh
>>>
>>> Talend Community Coder
>>> http://coders.talend.com
>>>
>>


-- 
Colm O hEigeartaigh

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


Go SDK build failures with maven

2018-04-19 Thread Colm O hEigeartaigh
Hi all,

Is the Apache Maven build still maintained? I'm seeing some recent failures
in the Go SDK:

[INFO] --- mvn-golang-wrapper:2.1.7:test (go-test) @ beam-sdks-go ---
[INFO] Prepared command line : bin/go test ./...
[ERROR]
[ERROR] -Exec.Err-
[ERROR] /home/colm/.mvnGoLang/.go_path/src/
cloud.google.com/go/pubsub/subscription.go:30:2: cannot find package "
golang.org/x/sync/errgroup" in any of:
[ERROR] /home/colm/.mvnGoLang/go1.9.linux-amd64/src/
golang.org/x/sync/errgroup (from $GOROOT)
[ERROR] /home/colm/.mvnGoLang/.go_path/src/golang.org/x/sync/errgroup
(from $GOPATH)
[ERROR] /home/colm/src/apache/beam/sdks/go/target/src/
golang.org/x/sync/errgroup
[ERROR] /home/colm/.mvnGoLang/.go_path/src/
cloud.google.com/go/pubsub/flow_controller.go:19:2: cannot find package "
golang.org/x/sync/semaphore" in any of:
[ERROR] /home/colm/.mvnGoLang/go1.9.linux-amd64/src/
golang.org/x/sync/semaphore (from $GOROOT)
[ERROR] /home/colm/.mvnGoLang/.go_path/src/golang.org/x/sync/semaphore
(from $GOPATH)
[ERROR] /home/colm/src/apache/beam/sdks/go/target/src/
golang.org/x/sync/semaphore
[ERROR]

Incidentally, when run via gradle I see:

/home/colm/src/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:75:
wrong number of args for format in Errorf call: 1 needed but 2 args
/home/colm/src/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:78:
wrong number of args for format in Errorf call: 1 needed but 2 args
/home/colm/src/apache/beam/sdks/go/target/src/
github.com/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:75: wrong
number of args for format in Errorf call: 1 needed but 2 args
/home/colm/src/apache/beam/sdks/go/target/src/
github.com/apache/beam/sdks/go/pkg/beam/util/pubsubx/pubsub.go:78: wrong
number of args for format in Errorf call: 1 needed but 2 args


Colm.


-- 
Colm O hEigeartaigh

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