Re: Serialization for org.apache.beam.sdk.util.WindowedValue$*

2016-06-02 Thread Amit Sela
Oh sorry, of course I meant Thomas Groh in my previous email.. But @Thomas
Weise this example

might
help, this is how the Spark runner uses Coders like Thomas Groh described.

And i agree that we should consider making PipelineOptions Serializable or
provide a generic solution for Runners.

Hope this helps,
Amit

On Thu, Jun 2, 2016 at 10:35 PM Amit Sela  wrote:

> Thomas is right, though in my case, I encountered this issue when using
> Spark's new API that uses Encoders
> 
>  not
> just for serialization but also for "translating" the object into a schema
> of optimized execution with Tungsten
> .
>
> I this case I'm using Kryo and I've solved this by registering (in Spark
> not Beam) custom serializers from
> https://github.com/magro/kryo-serializers
> I would consider (in the future) to implement Encoders with the help of
> Coders but I still didn't wrap my mind around this.
>
> On Thu, Jun 2, 2016 at 9:59 PM Thomas Groh 
> wrote:
>
>> The Beam Model ensures that all PCollections have a Coder; the PCollection
>> Coder is the standard way to materialize the elements of a
>> PCollection[1][2]. Most SDK-provided classes that will need to be
>> transferred across the wire have an associated coder, and some additional
>> default datatypes have coders associated with (in the CoderRegistry[3]).
>>
>> FullWindowedValueCoder[4] is capable of encoding and decoding the entirety
>> of a WindowedValue, and is constructed from a ValueCoder (obtained from
>> the
>> PCollection) and a WindowCoder (obtained from the WindowFn of the
>> WindowingStrategy of the PCollection). Given an input PCollection `pc`,
>> you
>> can construct the FullWindowedValueCoder with the following code snippet
>>
>> ```
>> FullWindowedValueCoder.of(pc.getCoder(),
>> pc.getWindowingStrategy().getWindowFn().windowCoder())
>> ```
>>
>> [1]
>>
>> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
>> [2]
>>
>> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java#L130
>> [3]
>>
>> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java#L94
>> [4]
>>
>> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java#L515
>>
>> On Thu, Jun 2, 2016 at 10:41 AM, Thomas Weise 
>> wrote:
>>
>> > Hi Amit,
>> >
>> > Thanks for the help. I implemented the same serialization workaround for
>> > the PipelineOptions. Since every distributed runner will have to solve
>> > this, would it make sense to provide the serialization support along
>> with
>> > the interface proxy?
>> >
>> > Here is the exception I get with with WindowedValue:
>> >
>> > com.esotericsoftware.kryo.KryoException: Class cannot be created
>> (missing
>> > no-arg constructor):
>> > org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow
>> > at
>> >
>> >
>> com.esotericsoftware.kryo.Kryo$DefaultInstantiatorStrategy.newInstantiatorOf(Kryo.java:1228)
>> > at com.esotericsoftware.kryo.Kryo.newInstantiator(Kryo.java:1049)
>> > at com.esotericsoftware.kryo.Kryo.newInstance(Kryo.java:1058)
>> > at
>> >
>> >
>> com.esotericsoftware.kryo.serializers.FieldSerializer.create(FieldSerializer.java:547)
>> > at
>> >
>> >
>> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:523)
>> > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:761)
>> >
>> > Thanks,
>> > Thomas
>> >
>> >
>> > On Wed, Jun 1, 2016 at 12:45 AM, Amit Sela 
>> wrote:
>> >
>> > > Hi Thomas,
>> > >
>> > > Spark and the Spark runner are using kryo for serialization and it
>> seems
>> > to
>> > > work just fine. What is your exact problem ? stack trace/message ?
>> > > I've hit an issue with Guava's ImmutableList/Map etc. and used
>> > > https://github.com/magro/kryo-serializers for that.
>> > >
>> > > For PipelineOptions you can take a look at the Spark runner code here:
>> > >
>> > >
>> >
>> https://github.com/apache/incubator-beam/blob/master/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java#L73
>> > >
>> > > I'd be happy to assist with Kryo.
>> > >
>> > > Thanks,
>> > > Amit
>> > >
>> > > On Wed, Jun 1, 2016 at 7:10 AM Thomas Weise  wrote:
>> > >
>> > > > Hi,
>> > > >
>> > > > I'm working on putting together a basic runner for Apache Apex.
>> > > >
>> > > > Hitting a couple of serialization related issues 

Re: Serialization for org.apache.beam.sdk.util.WindowedValue$*

2016-06-02 Thread Amit Sela
Thomas is right, though in my case, I encountered this issue when using
Spark's new API that uses Encoders

not
just for serialization but also for "translating" the object into a schema
of optimized execution with Tungsten
.

I this case I'm using Kryo and I've solved this by registering (in Spark
not Beam) custom serializers from https://github.com/magro/kryo-serializers
I would consider (in the future) to implement Encoders with the help of
Coders but I still didn't wrap my mind around this.

On Thu, Jun 2, 2016 at 9:59 PM Thomas Groh  wrote:

> The Beam Model ensures that all PCollections have a Coder; the PCollection
> Coder is the standard way to materialize the elements of a
> PCollection[1][2]. Most SDK-provided classes that will need to be
> transferred across the wire have an associated coder, and some additional
> default datatypes have coders associated with (in the CoderRegistry[3]).
>
> FullWindowedValueCoder[4] is capable of encoding and decoding the entirety
> of a WindowedValue, and is constructed from a ValueCoder (obtained from the
> PCollection) and a WindowCoder (obtained from the WindowFn of the
> WindowingStrategy of the PCollection). Given an input PCollection `pc`, you
> can construct the FullWindowedValueCoder with the following code snippet
>
> ```
> FullWindowedValueCoder.of(pc.getCoder(),
> pc.getWindowingStrategy().getWindowFn().windowCoder())
> ```
>
> [1]
>
> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
> [2]
>
> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java#L130
> [3]
>
> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java#L94
> [4]
>
> https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java#L515
>
> On Thu, Jun 2, 2016 at 10:41 AM, Thomas Weise 
> wrote:
>
> > Hi Amit,
> >
> > Thanks for the help. I implemented the same serialization workaround for
> > the PipelineOptions. Since every distributed runner will have to solve
> > this, would it make sense to provide the serialization support along with
> > the interface proxy?
> >
> > Here is the exception I get with with WindowedValue:
> >
> > com.esotericsoftware.kryo.KryoException: Class cannot be created (missing
> > no-arg constructor):
> > org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow
> > at
> >
> >
> com.esotericsoftware.kryo.Kryo$DefaultInstantiatorStrategy.newInstantiatorOf(Kryo.java:1228)
> > at com.esotericsoftware.kryo.Kryo.newInstantiator(Kryo.java:1049)
> > at com.esotericsoftware.kryo.Kryo.newInstance(Kryo.java:1058)
> > at
> >
> >
> com.esotericsoftware.kryo.serializers.FieldSerializer.create(FieldSerializer.java:547)
> > at
> >
> >
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:523)
> > at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:761)
> >
> > Thanks,
> > Thomas
> >
> >
> > On Wed, Jun 1, 2016 at 12:45 AM, Amit Sela  wrote:
> >
> > > Hi Thomas,
> > >
> > > Spark and the Spark runner are using kryo for serialization and it
> seems
> > to
> > > work just fine. What is your exact problem ? stack trace/message ?
> > > I've hit an issue with Guava's ImmutableList/Map etc. and used
> > > https://github.com/magro/kryo-serializers for that.
> > >
> > > For PipelineOptions you can take a look at the Spark runner code here:
> > >
> > >
> >
> https://github.com/apache/incubator-beam/blob/master/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java#L73
> > >
> > > I'd be happy to assist with Kryo.
> > >
> > > Thanks,
> > > Amit
> > >
> > > On Wed, Jun 1, 2016 at 7:10 AM Thomas Weise  wrote:
> > >
> > > > Hi,
> > > >
> > > > I'm working on putting together a basic runner for Apache Apex.
> > > >
> > > > Hitting a couple of serialization related issues with running tests.
> > Apex
> > > > is using Kryo for serialization by default (and Kryo can delegate to
> > > other
> > > > serialization frameworks).
> > > >
> > > > The inner classes of WindowedValue are private and have no default
> > > > constructor, which the Kryo field serializer does not like. Also
> these
> > > > classes are not Java serializable, so that's not a fallback option
> (not
> > > > that it would be efficient anyways).
> > > >
> > > > What's the recommended technique to move the WindowedValues over the
> > > wire?
> > > >
> > > > Also, PipelineOptions aren't serializable, while most other classes
> > are.
> > > > They are needed for example with 

Re: Serialization for org.apache.beam.sdk.util.WindowedValue$*

2016-06-02 Thread Thomas Groh
The Beam Model ensures that all PCollections have a Coder; the PCollection
Coder is the standard way to materialize the elements of a
PCollection[1][2]. Most SDK-provided classes that will need to be
transferred across the wire have an associated coder, and some additional
default datatypes have coders associated with (in the CoderRegistry[3]).

FullWindowedValueCoder[4] is capable of encoding and decoding the entirety
of a WindowedValue, and is constructed from a ValueCoder (obtained from the
PCollection) and a WindowCoder (obtained from the WindowFn of the
WindowingStrategy of the PCollection). Given an input PCollection `pc`, you
can construct the FullWindowedValueCoder with the following code snippet

```
FullWindowedValueCoder.of(pc.getCoder(),
pc.getWindowingStrategy().getWindowFn().windowCoder())
```

[1]
https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java
[2]
https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollection.java#L130
[3]
https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderRegistry.java#L94
[4]
https://github.com/apache/incubator-beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java#L515

On Thu, Jun 2, 2016 at 10:41 AM, Thomas Weise 
wrote:

> Hi Amit,
>
> Thanks for the help. I implemented the same serialization workaround for
> the PipelineOptions. Since every distributed runner will have to solve
> this, would it make sense to provide the serialization support along with
> the interface proxy?
>
> Here is the exception I get with with WindowedValue:
>
> com.esotericsoftware.kryo.KryoException: Class cannot be created (missing
> no-arg constructor):
> org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow
> at
>
> com.esotericsoftware.kryo.Kryo$DefaultInstantiatorStrategy.newInstantiatorOf(Kryo.java:1228)
> at com.esotericsoftware.kryo.Kryo.newInstantiator(Kryo.java:1049)
> at com.esotericsoftware.kryo.Kryo.newInstance(Kryo.java:1058)
> at
>
> com.esotericsoftware.kryo.serializers.FieldSerializer.create(FieldSerializer.java:547)
> at
>
> com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:523)
> at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:761)
>
> Thanks,
> Thomas
>
>
> On Wed, Jun 1, 2016 at 12:45 AM, Amit Sela  wrote:
>
> > Hi Thomas,
> >
> > Spark and the Spark runner are using kryo for serialization and it seems
> to
> > work just fine. What is your exact problem ? stack trace/message ?
> > I've hit an issue with Guava's ImmutableList/Map etc. and used
> > https://github.com/magro/kryo-serializers for that.
> >
> > For PipelineOptions you can take a look at the Spark runner code here:
> >
> >
> https://github.com/apache/incubator-beam/blob/master/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java#L73
> >
> > I'd be happy to assist with Kryo.
> >
> > Thanks,
> > Amit
> >
> > On Wed, Jun 1, 2016 at 7:10 AM Thomas Weise  wrote:
> >
> > > Hi,
> > >
> > > I'm working on putting together a basic runner for Apache Apex.
> > >
> > > Hitting a couple of serialization related issues with running tests.
> Apex
> > > is using Kryo for serialization by default (and Kryo can delegate to
> > other
> > > serialization frameworks).
> > >
> > > The inner classes of WindowedValue are private and have no default
> > > constructor, which the Kryo field serializer does not like. Also these
> > > classes are not Java serializable, so that's not a fallback option (not
> > > that it would be efficient anyways).
> > >
> > > What's the recommended technique to move the WindowedValues over the
> > wire?
> > >
> > > Also, PipelineOptions aren't serializable, while most other classes
> are.
> > > They are needed for example with DoFnRunnerBase, so what's the
> > recommended
> > > way to distribute them? Disassemble/reassemble? :)
> > >
> > > Thanks,
> > > Thomas
> > >
> >
>


Re: Build failed in Jenkins: beam_Release_NightlySnapshot #60

2016-06-02 Thread Jean-Baptiste Onofré

Gonna take a look as well.

Regards
JB

On 06/02/2016 06:14 PM, Davor Bonaci wrote:

New type of error; investigation in progress.

On Thu, Jun 2, 2016 at 12:30 AM, Apache Jenkins Server <
jenk...@builds.apache.org> wrote:


See 

Re: Serialization for org.apache.beam.sdk.util.WindowedValue$*

2016-06-02 Thread Thomas Weise
Hi Amit,

Thanks for the help. I implemented the same serialization workaround for
the PipelineOptions. Since every distributed runner will have to solve
this, would it make sense to provide the serialization support along with
the interface proxy?

Here is the exception I get with with WindowedValue:

com.esotericsoftware.kryo.KryoException: Class cannot be created (missing
no-arg constructor):
org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow
at
com.esotericsoftware.kryo.Kryo$DefaultInstantiatorStrategy.newInstantiatorOf(Kryo.java:1228)
at com.esotericsoftware.kryo.Kryo.newInstantiator(Kryo.java:1049)
at com.esotericsoftware.kryo.Kryo.newInstance(Kryo.java:1058)
at
com.esotericsoftware.kryo.serializers.FieldSerializer.create(FieldSerializer.java:547)
at
com.esotericsoftware.kryo.serializers.FieldSerializer.read(FieldSerializer.java:523)
at com.esotericsoftware.kryo.Kryo.readClassAndObject(Kryo.java:761)

Thanks,
Thomas


On Wed, Jun 1, 2016 at 12:45 AM, Amit Sela  wrote:

> Hi Thomas,
>
> Spark and the Spark runner are using kryo for serialization and it seems to
> work just fine. What is your exact problem ? stack trace/message ?
> I've hit an issue with Guava's ImmutableList/Map etc. and used
> https://github.com/magro/kryo-serializers for that.
>
> For PipelineOptions you can take a look at the Spark runner code here:
>
> https://github.com/apache/incubator-beam/blob/master/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java#L73
>
> I'd be happy to assist with Kryo.
>
> Thanks,
> Amit
>
> On Wed, Jun 1, 2016 at 7:10 AM Thomas Weise  wrote:
>
> > Hi,
> >
> > I'm working on putting together a basic runner for Apache Apex.
> >
> > Hitting a couple of serialization related issues with running tests. Apex
> > is using Kryo for serialization by default (and Kryo can delegate to
> other
> > serialization frameworks).
> >
> > The inner classes of WindowedValue are private and have no default
> > constructor, which the Kryo field serializer does not like. Also these
> > classes are not Java serializable, so that's not a fallback option (not
> > that it would be efficient anyways).
> >
> > What's the recommended technique to move the WindowedValues over the
> wire?
> >
> > Also, PipelineOptions aren't serializable, while most other classes are.
> > They are needed for example with DoFnRunnerBase, so what's the
> recommended
> > way to distribute them? Disassemble/reassemble? :)
> >
> > Thanks,
> > Thomas
> >
>


Re: Build failed in Jenkins: beam_Release_NightlySnapshot #60

2016-06-02 Thread Davor Bonaci
New type of error; investigation in progress.

On Thu, Jun 2, 2016 at 12:30 AM, Apache Jenkins Server <
jenk...@builds.apache.org> wrote:

> See  >
>
> Changes:
>
> [aljoscha.krettek] [BEAM-295] Remove erroneous close() calls in Flink
> Create Sources
>
> [bchambers] Forward port changes to GC holds
>
> [davor] Update pom.xml files formatting
>
> [davor] [maven-release-plugin] prepare branch release-0.1.0-incubating
>
> [davor] [maven-release-plugin] prepare for next development iteration
>
> [dhalperi] Use Structural Value keys instead of User Values
>
> --
> [...truncated 5106 lines...]
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
> longer than 100 characters.
> [WARNING] Entry:
> apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
> longer than 100 characters.
> [INFO] Building zip: <
> https://builds.apache.org/job/beam_Release_NightlySnapshot/ws/sdks/java/target/apache-beam-0.2.0-incubating-SNAPSHOT-src.zip
> >
> [INFO]
> [INFO] --- apache-rat-plugin:0.11:check (default) @ java-sdk-parent ---
> [INFO] 51 implicit excludes (use -debug for more details).
> [INFO] Exclude: **/target/**/*
> [INFO] 

Build failed in Jenkins: beam_Release_NightlySnapshot #60

2016-06-02 Thread Apache Jenkins Server
See 

Changes:

[aljoscha.krettek] [BEAM-295] Remove erroneous close() calls in Flink Create 
Sources

[bchambers] Forward port changes to GC holds

[davor] Update pom.xml files formatting

[davor] [maven-release-plugin] prepare branch release-0.1.0-incubating

[davor] [maven-release-plugin] prepare for next development iteration

[dhalperi] Use Structural Value keys instead of User Values

--
[...truncated 5106 lines...]
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/MapAggregatorValues.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/gcsfs/package-info.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/PropertyNames.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WeightedValue.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/CombineContextFactory.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ShardingWritableByteChannel.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersBitSet.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/IOChannelFactory.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/AssignWindows.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WindowingInternals.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/ExecutionContext.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/DirectModeExecutionContext.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/WindowTracing.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java
 longer than 100 characters.
[WARNING] Entry: 
apache-beam-0.2.0-incubating-SNAPSHOT/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java
 longer than 100 characters.
[INFO] Building zip: 

[INFO] 
[INFO] --- apache-rat-plugin:0.11:check (default) @ java-sdk-parent ---
[INFO] 51 implicit excludes (use -debug for more details).
[INFO] Exclude: **/target/**/*
[INFO] Exclude: **/dependency-reduced-pom.xml
[INFO] Exclude: .github/**/*
[INFO] Exclude: **/*.iml
[INFO] Exclude: **/package-list
[INFO] Exclude: **/user.avsc
[INFO] Exclude: **/test/resources/**/*.txt
[INFO] Exclude: **/test/**/.placeholder
[INFO] Exclude: .repository/**/*
[INFO] 700 resources 

Re: 0.1.0-incubating release

2016-06-02 Thread Jean-Baptiste Onofré

Another annoying thing is the main parent POM artifactId.

Now, it's just "parent". What do you think about renaming to "beam-parent" ?

Regarding the source distribution name, I would cancel this staging to 
fix that (I will have a PR ready soon).


Thoughts ?

Regards
JB

On 06/02/2016 03:46 AM, Davor Bonaci wrote:

Hi everyone!
We've started the release process for our first release, 0.1.0-incubating.

To recap previous discussions, we don't have particular functional goals
for this release. Instead, we'd like to make available what's currently in
the repository, as well as work through the release process.

With this in mind, we've:
* branched off the release branch [1] at master's commit 8485272,
* updated master to prepare for the second release, 0.2.0-incubating,
* built the first release candidate, RC1, and deployed it to a staging
repository [2].

We are not ready to start a vote just yet -- we've already identified a few
issues worth fixing. That said, I'd like to invite everybody to take a peek
and comment. I'm hoping we can address as many issues as possible before we
start the voting process.

Please let us know if you see any issues.

Thanks,
Davor

[1] https://github.com/apache/incubator-beam/tree/release-0.1.0-incubating
[2] https://repository.apache.org/content/repositories/orgapachebeam-1000/



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