[CANCELLED][VOTE] Release Apache Flink 1.0.0 (RC4)

2016-03-02 Thread Robert Metzger
This is the JIRA: https://issues.apache.org/jira/browse/FLINK-3565
It has been resolved by Max.

I'll merge the fix and create a new RC now

On Wed, Mar 2, 2016 at 12:15 PM, Aljoscha Krettek 
wrote:

> Hi,
> I saw this one when trying my job that was built against Scala 2.11:
>
> java.lang.NoClassDefFoundError: scala/collection/GenTraversableOnce$class
> at kafka.utils.Pool.(Pool.scala:28)
> at
> kafka.consumer.FetchRequestAndResponseStatsRegistry$.(FetchRequestAndResponseStats.scala:60)
> at
> kafka.consumer.FetchRequestAndResponseStatsRegistry$.(FetchRequestAndResponseStats.scala)
> at kafka.consumer.SimpleConsumer.(SimpleConsumer.scala:39)
> at
> kafka.javaapi.consumer.SimpleConsumer.(SimpleConsumer.scala:34)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.getPartitionsForTopic(FlinkKafkaConsumer08.java:518)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:218)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:193)
> at
> org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:160)
> at com.dataartisans.querywindow.WindowJob.main(WindowJob.java:93)
> at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
> at
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
> at
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
> at java.lang.reflect.Method.invoke(Method.java:606)
> at
> org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:505)
> at
> org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:403)
> at
> org.apache.flink.client.program.Client.runBlocking(Client.java:248)
> at
> org.apache.flink.client.CliFrontend.executeProgramBlocking(CliFrontend.java:866)
> at org.apache.flink.client.CliFrontend.run(CliFrontend.java:333)
> at
> org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1189)
> at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1239)
> Caused by: java.lang.ClassNotFoundException:
> scala.collection.GenTraversableOnce$class
> at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
> at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
> at java.security.AccessController.doPrivileged(Native Method)
> at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
> at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
> ... 21 more
>
> The job uses the Kafka 0.8 consumer. All deps have 2.11 suffix, the flink
> build is "Hadoop 2.7 Scala 2.11"
> > On 02 Mar 2016, at 11:36, Till Rohrmann  wrote:
> >
> > Do we continue using the google shared doc for RC3 for the release
> testing
> > coordination?
> >
> > On Wed, Mar 2, 2016 at 11:31 AM, Aljoscha Krettek 
> > wrote:
> >
> >> By the way, this is the commits that where added since rc3, so most of
> the
> >> testing that we already did should also be valid for this RC:
> >>
> >> $ git log origin/release-1.0.0-rc3..origin/release-1.0.0-rc4
> >> commit 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
> >> Author: Robert Metzger 
> >> Date:   Tue Mar 1 16:40:31 2016 +
> >>
> >>Commit for release 1.0.0
> >>
> >> commit 23dc2a4acf8e886384a66587ff393c2e62a69037
> >> Author: Stephan Ewen 
> >> Date:   Mon Feb 29 19:24:34 2016 +0100
> >>
> >>[FLINK-2788] [apis] Add TypeHint class to allow type-safe generic
> type
> >> parsing
> >>
> >>This closes #1744
> >>
> >> commit 43e5975d5426e22eb4ef90e0f468bd7f6cd35736
> >> Author: Stephan Ewen 
> >> Date:   Tue Mar 1 14:31:26 2016 +0100
> >>
> >>[FLINK-3554] [streaming] Emit a MAX Watermark after finite sources
> >> finished
> >>
> >>This closes #1750
> >>
> >> commit 8949ccf66b211b3c5cd8e66557afbff21fb093a6
> >> Author: Till Rohrmann 
> >> Date:   Tue Mar 1 12:36:22 2016 +0100
> >>
> >>[FLINK-3557] [stream, scala] Introduce secondary parameter list for
> >> fold function
> >>
> >>The fold API call takes an initial value as well as a fold function.
> >> In Scala it is possible
> >>to provide an anonymous function. In order to easily support multi
> >> line anonymous functions
> >>as well as being consistent with Scala's collection API, this PR adds
> >> another parameter list
> >>to the fold API call, which contains the fold function parameter.
> >>
> >>Insert spaces between first parameter list and curly braces of
> >> anonymous function
> >>
> >>This closes #1748.
> >>
> >> commit 

Re: Input type validation is killing me

2016-03-02 Thread Gyula Fóra
I opened this JIRA, if anyone has good examples, please add it in the
comments:
https://issues.apache.org/jira/browse/FLINK-3566

Gyula

Gyula Fóra  ezt írta (időpont: 2016. márc. 2., Sze,
15:54):

> Okay, I will open a JIRA issue
>
> Gyula
>
> Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
> 15:42):
>
>> Can you open an issue with an example of your custom TypeInfo? I will
>> then open a suitable PR for it.
>>
>>
>> On 02.03.2016 15:33, Gyula Fóra wrote:
>> > Would that work with generic classes?
>> >
>> > Timo Walther  ezt írta (időpont: 2016. márc. 2.,
>> Sze,
>> > 15:22):
>> >
>> >> After thinking about it, I think an even better solution is to provide
>> >> an interface for the TypeExtractor where the user can register mappings
>> >> from class to TypeInformation.
>> >> So that the TypeExctractor is more extensible. This would also solve
>> you
>> >> problem. What do you think?
>> >>
>> >> On 02.03.2016 15:00, Gyula Fóra wrote:
>> >>> Hi!
>> >>>
>> >>> Yes I think, that sounds good :) We just need to make sure that this
>> >> works
>> >>> with things like the TupleTypeInfo which is built-on but I can still
>> mix
>> >> in
>> >>> new Types for the fields.
>> >>>
>> >>>Thanks,
>> >>> Gyula
>> >>>
>> >>> Timo Walther  ezt írta (időpont: 2016. márc. 2.,
>> >> Sze,
>> >>> 14:02):
>> >>>
>>  The TypeExtractor's input type validation was designed for the
>> built-in
>>  TypeInformation classes.
>> 
>>  In your case of a new, unknown TypeInformation, the validation should
>>  simply skipped, because we can assume that you user knows what he is
>> >> doing.
>>  I can open a PR for that.
>> 
>> 
>>  On 02.03.2016 11:34, Aljoscha Krettek wrote:
>> > I think you have a point. Another user also just ran into problems
>> with
>>  the TypeExtractor. (The “Java Maps and TypeInformation” email).
>> > So let’s figure out what needs to be changed to make it work for all
>>  people.
>> > Cheers,
>> > Aljoscha
>> >> On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:
>> >>
>> >> Hey,
>> >>
>> >> I have brought up this issue a couple months back but I would like
>> to
>>  do it
>> >> again.
>> >>
>> >> I think the current way of validating the input type of udfs
>> against
>> >> the
>> >> out type of the preceeding operators is too aggressive and breaks a
>> >> lot
>>  of
>> >> code that should otherwise work.
>> >>
>> >> This issue appears all the time when I want to use my own
>> >> TypeInformations<> for operators such as creating my own Tuple
>> >> typeinfos
>> >> with custom types for the different fields and so.
>> >>
>> >> I have a more complex streaming job which would not run if I have
>> the
>>  input
>> >> type validation. Replacing the Exceptions with logging my Job runs
>> >> perfectly (making my point) but you can see the errors that would
>> have
>>  been
>> >> reported as exceptions in the logs:
>> >>
>> >> 2016-03-02 11:06:03,447 ERROR
>> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>>  Generic
>> >> object type ‘mypackage.TestEvent' expected but was
>> ‘mypackage.Event’.
>> >> 2016-03-02 11:06:03,450 ERROR
>> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>>  Unknown
>> >> Error. Type is null.
>> >> 2016-03-02 11:06:03,466 ERROR
>> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>>  Basic
>> >> type expected.
>> >> 2016-03-02 11:06:03,470 ERROR
>> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>>  Basic
>> >> type expected.
>> >>
>> >> Clearly all these errors where not valid in my case as my job runs
>> >> perfectly.
>> >>
>> >> Would it make sense to change the current behaviour or am I just
>> >> abusing
>> >> the .returns(..) and ResultTypeQueryable interfaces in unintended
>> >> ways.
>> >> Cheers,
>> >> Gyula
>> >>
>>
>>


Re: Input type validation is killing me

2016-03-02 Thread Gyula Fóra
Okay, I will open a JIRA issue

Gyula

Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
15:42):

> Can you open an issue with an example of your custom TypeInfo? I will
> then open a suitable PR for it.
>
>
> On 02.03.2016 15:33, Gyula Fóra wrote:
> > Would that work with generic classes?
> >
> > Timo Walther  ezt írta (időpont: 2016. márc. 2.,
> Sze,
> > 15:22):
> >
> >> After thinking about it, I think an even better solution is to provide
> >> an interface for the TypeExtractor where the user can register mappings
> >> from class to TypeInformation.
> >> So that the TypeExctractor is more extensible. This would also solve you
> >> problem. What do you think?
> >>
> >> On 02.03.2016 15:00, Gyula Fóra wrote:
> >>> Hi!
> >>>
> >>> Yes I think, that sounds good :) We just need to make sure that this
> >> works
> >>> with things like the TupleTypeInfo which is built-on but I can still
> mix
> >> in
> >>> new Types for the fields.
> >>>
> >>>Thanks,
> >>> Gyula
> >>>
> >>> Timo Walther  ezt írta (időpont: 2016. márc. 2.,
> >> Sze,
> >>> 14:02):
> >>>
>  The TypeExtractor's input type validation was designed for the
> built-in
>  TypeInformation classes.
> 
>  In your case of a new, unknown TypeInformation, the validation should
>  simply skipped, because we can assume that you user knows what he is
> >> doing.
>  I can open a PR for that.
> 
> 
>  On 02.03.2016 11:34, Aljoscha Krettek wrote:
> > I think you have a point. Another user also just ran into problems
> with
>  the TypeExtractor. (The “Java Maps and TypeInformation” email).
> > So let’s figure out what needs to be changed to make it work for all
>  people.
> > Cheers,
> > Aljoscha
> >> On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:
> >>
> >> Hey,
> >>
> >> I have brought up this issue a couple months back but I would like
> to
>  do it
> >> again.
> >>
> >> I think the current way of validating the input type of udfs against
> >> the
> >> out type of the preceeding operators is too aggressive and breaks a
> >> lot
>  of
> >> code that should otherwise work.
> >>
> >> This issue appears all the time when I want to use my own
> >> TypeInformations<> for operators such as creating my own Tuple
> >> typeinfos
> >> with custom types for the different fields and so.
> >>
> >> I have a more complex streaming job which would not run if I have
> the
>  input
> >> type validation. Replacing the Exceptions with logging my Job runs
> >> perfectly (making my point) but you can see the errors that would
> have
>  been
> >> reported as exceptions in the logs:
> >>
> >> 2016-03-02 11:06:03,447 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>  Generic
> >> object type ‘mypackage.TestEvent' expected but was
> ‘mypackage.Event’.
> >> 2016-03-02 11:06:03,450 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>  Unknown
> >> Error. Type is null.
> >> 2016-03-02 11:06:03,466 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>  Basic
> >> type expected.
> >> 2016-03-02 11:06:03,470 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
>  Basic
> >> type expected.
> >>
> >> Clearly all these errors where not valid in my case as my job runs
> >> perfectly.
> >>
> >> Would it make sense to change the current behaviour or am I just
> >> abusing
> >> the .returns(..) and ResultTypeQueryable interfaces in unintended
> >> ways.
> >> Cheers,
> >> Gyula
> >>
>
>


[jira] [Created] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-02 Thread Gyula Fora (JIRA)
Gyula Fora created FLINK-3566:
-

 Summary: Input type validation often fails on custom TypeInfo 
implementations
 Key: FLINK-3566
 URL: https://issues.apache.org/jira/browse/FLINK-3566
 Project: Flink
  Issue Type: Bug
  Components: Type Serialization System
Reporter: Gyula Fora


Input type validation often fails when used with custom type infos. One example 
of this behaviour can be reproduced by creating a custom type info with our own 
field type:

StreamExecutionEnvironment env = 
StreamExecutionEnvironment.getExecutionEnvironment();

env.generateSequence(1, 10).map(new MapFunction>() 
{
@Override
public Tuple1 map(Long value) throws 
Exception {
return Tuple1.of(Optional.of(value));
}
}).returns(new TupleTypeInfo<>(new 
OptionTypeInfo(BasicTypeInfo.LONG_TYPE_INFO)))
.keyBy(new KeySelector, 
Optional>() {

@Override
public Optional 
getKey(Tuple1 value) throws Exception {
return value.f0;
}
});

This will fail on Input type validation at the KeySelector (or any other 
function for example a mapper) with the following exception:

Input mismatch: Basic type expected.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Input type validation is killing me

2016-03-02 Thread Timo Walther
Can you open an issue with an example of your custom TypeInfo? I will 
then open a suitable PR for it.



On 02.03.2016 15:33, Gyula Fóra wrote:

Would that work with generic classes?

Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
15:22):


After thinking about it, I think an even better solution is to provide
an interface for the TypeExtractor where the user can register mappings
from class to TypeInformation.
So that the TypeExctractor is more extensible. This would also solve you
problem. What do you think?

On 02.03.2016 15:00, Gyula Fóra wrote:

Hi!

Yes I think, that sounds good :) We just need to make sure that this

works

with things like the TupleTypeInfo which is built-on but I can still mix

in

new Types for the fields.

   Thanks,
Gyula

Timo Walther  ezt írta (időpont: 2016. márc. 2.,

Sze,

14:02):


The TypeExtractor's input type validation was designed for the built-in
TypeInformation classes.

In your case of a new, unknown TypeInformation, the validation should
simply skipped, because we can assume that you user knows what he is

doing.

I can open a PR for that.


On 02.03.2016 11:34, Aljoscha Krettek wrote:

I think you have a point. Another user also just ran into problems with

the TypeExtractor. (The “Java Maps and TypeInformation” email).

So let’s figure out what needs to be changed to make it work for all

people.

Cheers,
Aljoscha

On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:

Hey,

I have brought up this issue a couple months back but I would like to

do it

again.

I think the current way of validating the input type of udfs against

the

out type of the preceeding operators is too aggressive and breaks a

lot

of

code that should otherwise work.

This issue appears all the time when I want to use my own
TypeInformations<> for operators such as creating my own Tuple

typeinfos

with custom types for the different fields and so.

I have a more complex streaming job which would not run if I have the

input

type validation. Replacing the Exceptions with logging my Job runs
perfectly (making my point) but you can see the errors that would have

been

reported as exceptions in the logs:

2016-03-02 11:06:03,447 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Generic

object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
2016-03-02 11:06:03,450 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Unknown

Error. Type is null.
2016-03-02 11:06:03,466 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Basic

type expected.
2016-03-02 11:06:03,470 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Basic

type expected.

Clearly all these errors where not valid in my case as my job runs
perfectly.

Would it make sense to change the current behaviour or am I just

abusing

the .returns(..) and ResultTypeQueryable interfaces in unintended

ways.

Cheers,
Gyula






Re: Input type validation is killing me

2016-03-02 Thread Gyula Fóra
Would that work with generic classes?

Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
15:22):

> After thinking about it, I think an even better solution is to provide
> an interface for the TypeExtractor where the user can register mappings
> from class to TypeInformation.
> So that the TypeExctractor is more extensible. This would also solve you
> problem. What do you think?
>
> On 02.03.2016 15:00, Gyula Fóra wrote:
> > Hi!
> >
> > Yes I think, that sounds good :) We just need to make sure that this
> works
> > with things like the TupleTypeInfo which is built-on but I can still mix
> in
> > new Types for the fields.
> >
> >   Thanks,
> > Gyula
> >
> > Timo Walther  ezt írta (időpont: 2016. márc. 2.,
> Sze,
> > 14:02):
> >
> >> The TypeExtractor's input type validation was designed for the built-in
> >> TypeInformation classes.
> >>
> >> In your case of a new, unknown TypeInformation, the validation should
> >> simply skipped, because we can assume that you user knows what he is
> doing.
> >> I can open a PR for that.
> >>
> >>
> >> On 02.03.2016 11:34, Aljoscha Krettek wrote:
> >>> I think you have a point. Another user also just ran into problems with
> >> the TypeExtractor. (The “Java Maps and TypeInformation” email).
> >>> So let’s figure out what needs to be changed to make it work for all
> >> people.
> >>> Cheers,
> >>> Aljoscha
>  On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:
> 
>  Hey,
> 
>  I have brought up this issue a couple months back but I would like to
> >> do it
>  again.
> 
>  I think the current way of validating the input type of udfs against
> the
>  out type of the preceeding operators is too aggressive and breaks a
> lot
> >> of
>  code that should otherwise work.
> 
>  This issue appears all the time when I want to use my own
>  TypeInformations<> for operators such as creating my own Tuple
> typeinfos
>  with custom types for the different fields and so.
> 
>  I have a more complex streaming job which would not run if I have the
> >> input
>  type validation. Replacing the Exceptions with logging my Job runs
>  perfectly (making my point) but you can see the errors that would have
> >> been
>  reported as exceptions in the logs:
> 
>  2016-03-02 11:06:03,447 ERROR
>  org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> >> Generic
>  object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
>  2016-03-02 11:06:03,450 ERROR
>  org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> >> Unknown
>  Error. Type is null.
>  2016-03-02 11:06:03,466 ERROR
>  org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> >> Basic
>  type expected.
>  2016-03-02 11:06:03,470 ERROR
>  org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> >> Basic
>  type expected.
> 
>  Clearly all these errors where not valid in my case as my job runs
>  perfectly.
> 
>  Would it make sense to change the current behaviour or am I just
> abusing
>  the .returns(..) and ResultTypeQueryable interfaces in unintended
> ways.
> 
>  Cheers,
>  Gyula
> >>
>
>


Re: Input type validation is killing me

2016-03-02 Thread Timo Walther
After thinking about it, I think an even better solution is to provide 
an interface for the TypeExtractor where the user can register mappings 
from class to TypeInformation.
So that the TypeExctractor is more extensible. This would also solve you 
problem. What do you think?


On 02.03.2016 15:00, Gyula Fóra wrote:

Hi!

Yes I think, that sounds good :) We just need to make sure that this works
with things like the TupleTypeInfo which is built-on but I can still mix in
new Types for the fields.

  Thanks,
Gyula

Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
14:02):


The TypeExtractor's input type validation was designed for the built-in
TypeInformation classes.

In your case of a new, unknown TypeInformation, the validation should
simply skipped, because we can assume that you user knows what he is doing.
I can open a PR for that.


On 02.03.2016 11:34, Aljoscha Krettek wrote:

I think you have a point. Another user also just ran into problems with

the TypeExtractor. (The “Java Maps and TypeInformation” email).

So let’s figure out what needs to be changed to make it work for all

people.

Cheers,
Aljoscha

On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:

Hey,

I have brought up this issue a couple months back but I would like to

do it

again.

I think the current way of validating the input type of udfs against the
out type of the preceeding operators is too aggressive and breaks a lot

of

code that should otherwise work.

This issue appears all the time when I want to use my own
TypeInformations<> for operators such as creating my own Tuple typeinfos
with custom types for the different fields and so.

I have a more complex streaming job which would not run if I have the

input

type validation. Replacing the Exceptions with logging my Job runs
perfectly (making my point) but you can see the errors that would have

been

reported as exceptions in the logs:

2016-03-02 11:06:03,447 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Generic

object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
2016-03-02 11:06:03,450 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Unknown

Error. Type is null.
2016-03-02 11:06:03,466 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Basic

type expected.
2016-03-02 11:06:03,470 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:

Basic

type expected.

Clearly all these errors where not valid in my case as my job runs
perfectly.

Would it make sense to change the current behaviour or am I just abusing
the .returns(..) and ResultTypeQueryable interfaces in unintended ways.

Cheers,
Gyula






Re: Input type validation is killing me

2016-03-02 Thread Gyula Fóra
Hi!

Yes I think, that sounds good :) We just need to make sure that this works
with things like the TupleTypeInfo which is built-on but I can still mix in
new Types for the fields.

 Thanks,
Gyula

Timo Walther  ezt írta (időpont: 2016. márc. 2., Sze,
14:02):

> The TypeExtractor's input type validation was designed for the built-in
> TypeInformation classes.
>
> In your case of a new, unknown TypeInformation, the validation should
> simply skipped, because we can assume that you user knows what he is doing.
> I can open a PR for that.
>
>
> On 02.03.2016 11:34, Aljoscha Krettek wrote:
> > I think you have a point. Another user also just ran into problems with
> the TypeExtractor. (The “Java Maps and TypeInformation” email).
> >
> > So let’s figure out what needs to be changed to make it work for all
> people.
> >
> > Cheers,
> > Aljoscha
> >> On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:
> >>
> >> Hey,
> >>
> >> I have brought up this issue a couple months back but I would like to
> do it
> >> again.
> >>
> >> I think the current way of validating the input type of udfs against the
> >> out type of the preceeding operators is too aggressive and breaks a lot
> of
> >> code that should otherwise work.
> >>
> >> This issue appears all the time when I want to use my own
> >> TypeInformations<> for operators such as creating my own Tuple typeinfos
> >> with custom types for the different fields and so.
> >>
> >> I have a more complex streaming job which would not run if I have the
> input
> >> type validation. Replacing the Exceptions with logging my Job runs
> >> perfectly (making my point) but you can see the errors that would have
> been
> >> reported as exceptions in the logs:
> >>
> >> 2016-03-02 11:06:03,447 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> Generic
> >> object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
> >> 2016-03-02 11:06:03,450 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> Unknown
> >> Error. Type is null.
> >> 2016-03-02 11:06:03,466 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> Basic
> >> type expected.
> >> 2016-03-02 11:06:03,470 ERROR
> >> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch:
> Basic
> >> type expected.
> >>
> >> Clearly all these errors where not valid in my case as my job runs
> >> perfectly.
> >>
> >> Would it make sense to change the current behaviour or am I just abusing
> >> the .returns(..) and ResultTypeQueryable interfaces in unintended ways.
> >>
> >> Cheers,
> >> Gyula
>
>


[jira] [Created] (FLINK-3564) Implement distinct() for Table API

2016-03-02 Thread Timo Walther (JIRA)
Timo Walther created FLINK-3564:
---

 Summary: Implement distinct() for Table API
 Key: FLINK-3564
 URL: https://issues.apache.org/jira/browse/FLINK-3564
 Project: Flink
  Issue Type: New Feature
  Components: Table API
Reporter: Timo Walther
Assignee: Timo Walther
Priority: Minor


This is only syntactic sugar for grouping of all fields.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Created] (FLINK-3563) .returns() doesn't compile when using .map() with a custom MapFunction

2016-03-02 Thread Simone Robutti (JIRA)
Simone Robutti created FLINK-3563:
-

 Summary: .returns() doesn't compile when using .map() with a 
custom MapFunction
 Key: FLINK-3563
 URL: https://issues.apache.org/jira/browse/FLINK-3563
 Project: Flink
  Issue Type: Bug
  Components: Type Serialization System
Affects Versions: 0.10.1
Reporter: Simone Robutti
Priority: Minor


Defined a DummyMapFunction that goes from a java Map to another java Map like 
this:

{code:title=DummyMapFunction.scalaborderStyle=solid}
class DummyMapFunction() extends MapFunction[java.util.Map[String, Any], 
java.util.Map[FieldName, Any]] {
  override def map(input: java.util.Map[String, Any]): java.util.Map[FieldName, 
Any] = {
val result: java.util.Map[FieldName, Any] = new 
java.util.HashMap[FieldName, Any]()
result
  }
}
{code}

and trying to use it with a map:

{code:title=Main.java}
DummyMapFunction operator = new DummyMapFunction();

DataSource> dataset = env.fromCollection(input);
List> collectedResult = 
dataset.map(operator).returns(java.util.Map.class).collect();
{code}

the returns call doesn't compile because it can't resolve the returns method 
with the parameter.

But if insted of creating a variable of type DummyMapFunction I create a

{code}
MapFunction operator=new DummyMapFuction();
{code}

or I explicitly cast the variable to a MapFunction, it compiles and work 
flawlessly.

This is a trick that works but I think is an unexpected behaviour. 



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [VOTE] Release Apache Flink 1.0.0 (RC4)

2016-03-02 Thread Aljoscha Krettek
Hi,
I saw this one when trying my job that was built against Scala 2.11:

java.lang.NoClassDefFoundError: scala/collection/GenTraversableOnce$class
at kafka.utils.Pool.(Pool.scala:28)
at 
kafka.consumer.FetchRequestAndResponseStatsRegistry$.(FetchRequestAndResponseStats.scala:60)
at 
kafka.consumer.FetchRequestAndResponseStatsRegistry$.(FetchRequestAndResponseStats.scala)
at kafka.consumer.SimpleConsumer.(SimpleConsumer.scala:39)
at kafka.javaapi.consumer.SimpleConsumer.(SimpleConsumer.scala:34)
at 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.getPartitionsForTopic(FlinkKafkaConsumer08.java:518)
at 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:218)
at 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:193)
at 
org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08.(FlinkKafkaConsumer08.java:160)
at com.dataartisans.querywindow.WindowJob.main(WindowJob.java:93)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
at java.lang.reflect.Method.invoke(Method.java:606)
at 
org.apache.flink.client.program.PackagedProgram.callMainMethod(PackagedProgram.java:505)
at 
org.apache.flink.client.program.PackagedProgram.invokeInteractiveModeForExecution(PackagedProgram.java:403)
at org.apache.flink.client.program.Client.runBlocking(Client.java:248)
at 
org.apache.flink.client.CliFrontend.executeProgramBlocking(CliFrontend.java:866)
at org.apache.flink.client.CliFrontend.run(CliFrontend.java:333)
at 
org.apache.flink.client.CliFrontend.parseParameters(CliFrontend.java:1189)
at org.apache.flink.client.CliFrontend.main(CliFrontend.java:1239)
Caused by: java.lang.ClassNotFoundException: 
scala.collection.GenTraversableOnce$class
at java.net.URLClassLoader$1.run(URLClassLoader.java:366)
at java.net.URLClassLoader$1.run(URLClassLoader.java:355)
at java.security.AccessController.doPrivileged(Native Method)
at java.net.URLClassLoader.findClass(URLClassLoader.java:354)
at java.lang.ClassLoader.loadClass(ClassLoader.java:425)
at java.lang.ClassLoader.loadClass(ClassLoader.java:358)
... 21 more

The job uses the Kafka 0.8 consumer. All deps have 2.11 suffix, the flink build 
is "Hadoop 2.7 Scala 2.11"
> On 02 Mar 2016, at 11:36, Till Rohrmann  wrote:
> 
> Do we continue using the google shared doc for RC3 for the release testing
> coordination?
> 
> On Wed, Mar 2, 2016 at 11:31 AM, Aljoscha Krettek 
> wrote:
> 
>> By the way, this is the commits that where added since rc3, so most of the
>> testing that we already did should also be valid for this RC:
>> 
>> $ git log origin/release-1.0.0-rc3..origin/release-1.0.0-rc4
>> commit 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
>> Author: Robert Metzger 
>> Date:   Tue Mar 1 16:40:31 2016 +
>> 
>>Commit for release 1.0.0
>> 
>> commit 23dc2a4acf8e886384a66587ff393c2e62a69037
>> Author: Stephan Ewen 
>> Date:   Mon Feb 29 19:24:34 2016 +0100
>> 
>>[FLINK-2788] [apis] Add TypeHint class to allow type-safe generic type
>> parsing
>> 
>>This closes #1744
>> 
>> commit 43e5975d5426e22eb4ef90e0f468bd7f6cd35736
>> Author: Stephan Ewen 
>> Date:   Tue Mar 1 14:31:26 2016 +0100
>> 
>>[FLINK-3554] [streaming] Emit a MAX Watermark after finite sources
>> finished
>> 
>>This closes #1750
>> 
>> commit 8949ccf66b211b3c5cd8e66557afbff21fb093a6
>> Author: Till Rohrmann 
>> Date:   Tue Mar 1 12:36:22 2016 +0100
>> 
>>[FLINK-3557] [stream, scala] Introduce secondary parameter list for
>> fold function
>> 
>>The fold API call takes an initial value as well as a fold function.
>> In Scala it is possible
>>to provide an anonymous function. In order to easily support multi
>> line anonymous functions
>>as well as being consistent with Scala's collection API, this PR adds
>> another parameter list
>>to the fold API call, which contains the fold function parameter.
>> 
>>Insert spaces between first parameter list and curly braces of
>> anonymous function
>> 
>>This closes #1748.
>> 
>> commit 2d56081e29996f3f83e1a882151c06e44233d38f
>> Author: Ufuk Celebi 
>> Date:   Tue Mar 1 14:58:01 2016 +0100
>> 
>>[FLINK-3559] [dist] Don't print INFO if no active process
>> 
>>This closes #1751.
>> 
>> commit 6262a0edde07f3ca968f88814e25927be7ed07c2
>> Author: Ufuk Celebi 
>> Date:   Tue Mar 1 12:23:31 2016 +0100
>> 
>>[FLINK-3556] [runtime] Remove false check in HA blob store
>> configuration

Re: Input type validation is killing me

2016-03-02 Thread Aljoscha Krettek
I think you have a point. Another user also just ran into problems with the 
TypeExtractor. (The “Java Maps and TypeInformation” email).

So let’s figure out what needs to be changed to make it work for all people.

Cheers,
Aljoscha
> On 02 Mar 2016, at 11:15, Gyula Fóra  wrote:
> 
> Hey,
> 
> I have brought up this issue a couple months back but I would like to do it
> again.
> 
> I think the current way of validating the input type of udfs against the
> out type of the preceeding operators is too aggressive and breaks a lot of
> code that should otherwise work.
> 
> This issue appears all the time when I want to use my own
> TypeInformations<> for operators such as creating my own Tuple typeinfos
> with custom types for the different fields and so.
> 
> I have a more complex streaming job which would not run if I have the input
> type validation. Replacing the Exceptions with logging my Job runs
> perfectly (making my point) but you can see the errors that would have been
> reported as exceptions in the logs:
> 
> 2016-03-02 11:06:03,447 ERROR
> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Generic
> object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
> 2016-03-02 11:06:03,450 ERROR
> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Unknown
> Error. Type is null.
> 2016-03-02 11:06:03,466 ERROR
> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Basic
> type expected.
> 2016-03-02 11:06:03,470 ERROR
> org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Basic
> type expected.
> 
> Clearly all these errors where not valid in my case as my job runs
> perfectly.
> 
> Would it make sense to change the current behaviour or am I just abusing
> the .returns(..) and ResultTypeQueryable interfaces in unintended ways.
> 
> Cheers,
> Gyula



[jira] [Created] (FLINK-3562) Update docs in the course of EventTimeSourceFunction removal

2016-03-02 Thread Maximilian Michels (JIRA)
Maximilian Michels created FLINK-3562:
-

 Summary: Update docs in the course of EventTimeSourceFunction 
removal
 Key: FLINK-3562
 URL: https://issues.apache.org/jira/browse/FLINK-3562
 Project: Flink
  Issue Type: Bug
Affects Versions: 1.0.0
Reporter: Maximilian Michels
Assignee: Maximilian Michels
Priority: Minor
 Fix For: 1.0.0, 1.1.0


EventTimeSourceFunction has been removed. Documentation and JavaDocs haven't 
been updated.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: [VOTE] Release Apache Flink 1.0.0 (RC4)

2016-03-02 Thread Aljoscha Krettek
By the way, this is the commits that where added since rc3, so most of the 
testing that we already did should also be valid for this RC:

$ git log origin/release-1.0.0-rc3..origin/release-1.0.0-rc4
commit 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
Author: Robert Metzger 
Date:   Tue Mar 1 16:40:31 2016 +

Commit for release 1.0.0

commit 23dc2a4acf8e886384a66587ff393c2e62a69037
Author: Stephan Ewen 
Date:   Mon Feb 29 19:24:34 2016 +0100

[FLINK-2788] [apis] Add TypeHint class to allow type-safe generic type 
parsing

This closes #1744

commit 43e5975d5426e22eb4ef90e0f468bd7f6cd35736
Author: Stephan Ewen 
Date:   Tue Mar 1 14:31:26 2016 +0100

[FLINK-3554] [streaming] Emit a MAX Watermark after finite sources finished

This closes #1750

commit 8949ccf66b211b3c5cd8e66557afbff21fb093a6
Author: Till Rohrmann 
Date:   Tue Mar 1 12:36:22 2016 +0100

[FLINK-3557] [stream, scala] Introduce secondary parameter list for fold 
function

The fold API call takes an initial value as well as a fold function. In 
Scala it is possible
to provide an anonymous function. In order to easily support multi line 
anonymous functions
as well as being consistent with Scala's collection API, this PR adds 
another parameter list
to the fold API call, which contains the fold function parameter.

Insert spaces between first parameter list and curly braces of anonymous 
function

This closes #1748.

commit 2d56081e29996f3f83e1a882151c06e44233d38f
Author: Ufuk Celebi 
Date:   Tue Mar 1 14:58:01 2016 +0100

[FLINK-3559] [dist] Don't print INFO if no active process

This closes #1751.

commit 6262a0edde07f3ca968f88814e25927be7ed07c2
Author: Ufuk Celebi 
Date:   Tue Mar 1 12:23:31 2016 +0100

[FLINK-3556] [runtime] Remove false check in HA blob store configuration

This closes #1749.

commit 8e30f86657e4432a226d28810bac54cdcc906c04
Author: vasia 
Date:   Mon Feb 29 22:49:35 2016 +0100

[docs] fix readme typos; use the same scala style in the examples

This closes #1743

> On 02 Mar 2016, at 10:26, Robert Metzger  wrote:
> 
> The release binaries are now located here:
> http://home.apache.org/~rmetzger/flink-1.0.0-rc4/
> 
> 
> On Wed, Mar 2, 2016 at 10:16 AM, Robert Metzger  wrote:
> 
>> Yes, There was an email from Infra that they are going to shut down
>> people.apache.org on March 1.
>> I'll try to move the binaries to the new server ("home.apache.org").
>> 
>> On Wed, Mar 2, 2016 at 9:27 AM, Ufuk Celebi  wrote:
>> 
>>> I get a 404 for the binaries. It's a INFRA thing I guess, because my
>>> personal apache user page is also down/gone. :-(
>>> 
>>> On Tue, Mar 1, 2016 at 10:42 PM, Robert Metzger 
>>> wrote:
 Dear Flink community,
 
 Please vote on releasing the following candidate as Apache Flink
>>> version 1.0
 .0.
 
 This is the fourth RC.
 Here is a document to report on the testing and release verification:
 
>>> https://docs.google.com/document/d/1hoQ5k4WQteNj2OoPwpQPD4ZVHrCwM1pTlUVww8ld7oY/edit#heading=h.2v6zy51pgj33
 
 
 The commit to be voted on (
 http://git-wip-us.apache.org/repos/asf/flink/commit/1b0a8c4e)
 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
 
 Branch:
 release-1.0.0-rc4 (see
 
>>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc4
 )
 
 The release artifacts to be voted on can be found at:
 http://people.apache.org/~rmetzger/flink-1.0.0-rc4/
 
 The release artifacts are signed with the key with fingerprint D9839159:
 http://www.apache.org/dist/flink/KEYS
 
 The staging repository for this release can be found at:
 https://repository.apache.org/content/repositories/orgapacheflink-1066
 
 -
 
 The vote is open until Friday and passes if a majority of at least three
 +1 PMC votes are cast.
 
 The vote ends on Friday, March 4, 23:00 CET.
 
 [ ] +1 Release this package as Apache Flink 1.0.0
 [ ] -1 Do not release this package because ...
>>> 
>> 
>> 



Re: Apache Big Data

2016-03-02 Thread Till Rohrmann
That is cool Nikolaas :-) Looking forward to the scala-shell for streaming
:-)

On Wed, Mar 2, 2016 at 10:53 AM, Nikolaas s 
wrote:

> Hi guys,
>
> I've integrated streaming in zeppelin for flink.
> It works using the scala shell, which I extended to support the streaming
> application.
> Unfortunately the scala-shell for streaming is not yet included in the
> Flink-master, and changed a bit upon public request since I implemented the
> zeppelin-flink connection.
> It should, however, work with these two versions of Flink and Zeppelin:
>
>
> https://github.com/nikste/flink/tree/Flink-2522_Scala_shell_streaming_download_linux
> https://github.com/nikste/incubator-zeppelin/tree/visualizationDemo
>
> Note that the resubmission of streaming jobs by zeppelin is not yet
> working.
> The running streaming job has to be cancelled by zeppelin, which is
> possible by finding out the job-id and sending a cancel message to the task
> manager.
> I'll continue integrating it with the current versions of Flink and
> Zeppelin as soon as I have the time.
>
> cheers,
> Nik
>
>
> 2016-03-02 10:27 GMT+01:00 Till Rohrmann :
>
> > Great to hear that you two are giving a talk at ApacheCon.
> >
> > As far as I know, there's nobody working on a streaming interpreter for
> > Zeppelin. People thought about doing it, but it never got realized so
> far.
> > But I think it should not be too difficult to implement. So if you wanna
> > take the lead there, go ahead. I'm convinced that the community would be
> > very happy about it.
> >
> > Cheers,
> > Till
> >
> > On Tue, Mar 1, 2016 at 9:15 PM, Suneel Marthi 
> wrote:
> >
> > > Same here, will be presenting Distributed ML using Mahout on Flink at
> > > ApacheCon.
> > >
> > > Some big shout outs to Flink coming in Vancouver.
> > >
> > > On Tue, Mar 1, 2016 at 2:35 PM, Trevor Grant  >
> > > wrote:
> > >
> > > > Any one who is attending Apache: Big Data in Vancouver May 9-12 stop
> by
> > > and
> > > > check out my talk: *Everyone Plays: Collaborative Data Science With
> > > > Zeppelin*
> > > >
> > > > I'll obviously be giving some big shout outs to Flink.
> > > >
> > > > Which leads me to my next question: is anyone working on a Flink
> > > Streaming
> > > > Interpreter for Zeppelin? I've been heads down on writing those LTSM
> > > Neural
> > > > Nets for Flink (also hoping to have those ready for show casing in
> > time),
> > > > but a notebook running Flink Streaming would be quite dope.
> > > >
> > > > Thanks!
> > > > tg
> > > >
> > > >
> > > > Trevor Grant
> > > > Data Scientist
> > > > https://github.com/rawkintrevo
> > > > http://stackexchange.com/users/3002022/rawkintrevo
> > > > http://trevorgrant.org
> > > >
> > > > *"Fortunate is he, who is able to know the causes of things."
> -Virgil*
> > > >
> > >
> >
>


Input type validation is killing me

2016-03-02 Thread Gyula Fóra
Hey,

I have brought up this issue a couple months back but I would like to do it
again.

I think the current way of validating the input type of udfs against the
out type of the preceeding operators is too aggressive and breaks a lot of
code that should otherwise work.

This issue appears all the time when I want to use my own
TypeInformations<> for operators such as creating my own Tuple typeinfos
with custom types for the different fields and so.

I have a more complex streaming job which would not run if I have the input
type validation. Replacing the Exceptions with logging my Job runs
perfectly (making my point) but you can see the errors that would have been
reported as exceptions in the logs:

2016-03-02 11:06:03,447 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Generic
object type ‘mypackage.TestEvent' expected but was ‘mypackage.Event’.
2016-03-02 11:06:03,450 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Unknown
Error. Type is null.
2016-03-02 11:06:03,466 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Basic
type expected.
2016-03-02 11:06:03,470 ERROR
org.apache.flink.api.java.typeutils.TypeExtractor - Input mismatch: Basic
type expected.

Clearly all these errors where not valid in my case as my job runs
perfectly.

Would it make sense to change the current behaviour or am I just abusing
the .returns(..) and ResultTypeQueryable interfaces in unintended ways.

Cheers,
Gyula


Re: Apache Big Data

2016-03-02 Thread Nikolaas s
Hi guys,

I've integrated streaming in zeppelin for flink.
It works using the scala shell, which I extended to support the streaming
application.
Unfortunately the scala-shell for streaming is not yet included in the
Flink-master, and changed a bit upon public request since I implemented the
zeppelin-flink connection.
It should, however, work with these two versions of Flink and Zeppelin:

https://github.com/nikste/flink/tree/Flink-2522_Scala_shell_streaming_download_linux
https://github.com/nikste/incubator-zeppelin/tree/visualizationDemo

Note that the resubmission of streaming jobs by zeppelin is not yet working.
The running streaming job has to be cancelled by zeppelin, which is
possible by finding out the job-id and sending a cancel message to the task
manager.
I'll continue integrating it with the current versions of Flink and
Zeppelin as soon as I have the time.

cheers,
Nik


2016-03-02 10:27 GMT+01:00 Till Rohrmann :

> Great to hear that you two are giving a talk at ApacheCon.
>
> As far as I know, there's nobody working on a streaming interpreter for
> Zeppelin. People thought about doing it, but it never got realized so far.
> But I think it should not be too difficult to implement. So if you wanna
> take the lead there, go ahead. I'm convinced that the community would be
> very happy about it.
>
> Cheers,
> Till
>
> On Tue, Mar 1, 2016 at 9:15 PM, Suneel Marthi  wrote:
>
> > Same here, will be presenting Distributed ML using Mahout on Flink at
> > ApacheCon.
> >
> > Some big shout outs to Flink coming in Vancouver.
> >
> > On Tue, Mar 1, 2016 at 2:35 PM, Trevor Grant 
> > wrote:
> >
> > > Any one who is attending Apache: Big Data in Vancouver May 9-12 stop by
> > and
> > > check out my talk: *Everyone Plays: Collaborative Data Science With
> > > Zeppelin*
> > >
> > > I'll obviously be giving some big shout outs to Flink.
> > >
> > > Which leads me to my next question: is anyone working on a Flink
> > Streaming
> > > Interpreter for Zeppelin? I've been heads down on writing those LTSM
> > Neural
> > > Nets for Flink (also hoping to have those ready for show casing in
> time),
> > > but a notebook running Flink Streaming would be quite dope.
> > >
> > > Thanks!
> > > tg
> > >
> > >
> > > Trevor Grant
> > > Data Scientist
> > > https://github.com/rawkintrevo
> > > http://stackexchange.com/users/3002022/rawkintrevo
> > > http://trevorgrant.org
> > >
> > > *"Fortunate is he, who is able to know the causes of things."  -Virgil*
> > >
> >
>


[jira] [Created] (FLINK-3561) ExecutionConfig's timestampsEnabled is unused

2016-03-02 Thread Maximilian Michels (JIRA)
Maximilian Michels created FLINK-3561:
-

 Summary: ExecutionConfig's timestampsEnabled is unused
 Key: FLINK-3561
 URL: https://issues.apache.org/jira/browse/FLINK-3561
 Project: Flink
  Issue Type: Bug
  Components: Streaming
Affects Versions: 1.0.0, 1.1.0
Reporter: Maximilian Michels
Assignee: Maximilian Michels
Priority: Minor
 Fix For: 1.1.0


Seems like the flag can be removed. What do you think [~aljoscha]?



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


Re: Apache Big Data

2016-03-02 Thread Till Rohrmann
Great to hear that you two are giving a talk at ApacheCon.

As far as I know, there's nobody working on a streaming interpreter for
Zeppelin. People thought about doing it, but it never got realized so far.
But I think it should not be too difficult to implement. So if you wanna
take the lead there, go ahead. I'm convinced that the community would be
very happy about it.

Cheers,
Till

On Tue, Mar 1, 2016 at 9:15 PM, Suneel Marthi  wrote:

> Same here, will be presenting Distributed ML using Mahout on Flink at
> ApacheCon.
>
> Some big shout outs to Flink coming in Vancouver.
>
> On Tue, Mar 1, 2016 at 2:35 PM, Trevor Grant 
> wrote:
>
> > Any one who is attending Apache: Big Data in Vancouver May 9-12 stop by
> and
> > check out my talk: *Everyone Plays: Collaborative Data Science With
> > Zeppelin*
> >
> > I'll obviously be giving some big shout outs to Flink.
> >
> > Which leads me to my next question: is anyone working on a Flink
> Streaming
> > Interpreter for Zeppelin? I've been heads down on writing those LTSM
> Neural
> > Nets for Flink (also hoping to have those ready for show casing in time),
> > but a notebook running Flink Streaming would be quite dope.
> >
> > Thanks!
> > tg
> >
> >
> > Trevor Grant
> > Data Scientist
> > https://github.com/rawkintrevo
> > http://stackexchange.com/users/3002022/rawkintrevo
> > http://trevorgrant.org
> >
> > *"Fortunate is he, who is able to know the causes of things."  -Virgil*
> >
>


Re: [VOTE] Release Apache Flink 1.0.0 (RC4)

2016-03-02 Thread Robert Metzger
The release binaries are now located here:
http://home.apache.org/~rmetzger/flink-1.0.0-rc4/


On Wed, Mar 2, 2016 at 10:16 AM, Robert Metzger  wrote:

> Yes, There was an email from Infra that they are going to shut down
> people.apache.org on March 1.
> I'll try to move the binaries to the new server ("home.apache.org").
>
> On Wed, Mar 2, 2016 at 9:27 AM, Ufuk Celebi  wrote:
>
>> I get a 404 for the binaries. It's a INFRA thing I guess, because my
>> personal apache user page is also down/gone. :-(
>>
>> On Tue, Mar 1, 2016 at 10:42 PM, Robert Metzger 
>> wrote:
>> > Dear Flink community,
>> >
>> > Please vote on releasing the following candidate as Apache Flink
>> version 1.0
>> > .0.
>> >
>> > This is the fourth RC.
>> > Here is a document to report on the testing and release verification:
>> >
>> https://docs.google.com/document/d/1hoQ5k4WQteNj2OoPwpQPD4ZVHrCwM1pTlUVww8ld7oY/edit#heading=h.2v6zy51pgj33
>> >
>> >
>> > The commit to be voted on (
>> > http://git-wip-us.apache.org/repos/asf/flink/commit/1b0a8c4e)
>> > 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
>> >
>> > Branch:
>> > release-1.0.0-rc4 (see
>> >
>> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc4
>> > )
>> >
>> > The release artifacts to be voted on can be found at:
>> > http://people.apache.org/~rmetzger/flink-1.0.0-rc4/
>> >
>> > The release artifacts are signed with the key with fingerprint D9839159:
>> > http://www.apache.org/dist/flink/KEYS
>> >
>> > The staging repository for this release can be found at:
>> > https://repository.apache.org/content/repositories/orgapacheflink-1066
>> >
>> > -
>> >
>> > The vote is open until Friday and passes if a majority of at least three
>> > +1 PMC votes are cast.
>> >
>> > The vote ends on Friday, March 4, 23:00 CET.
>> >
>> > [ ] +1 Release this package as Apache Flink 1.0.0
>> > [ ] -1 Do not release this package because ...
>>
>
>


Re: [VOTE] Release Apache Flink 1.0.0 (RC4)

2016-03-02 Thread Ufuk Celebi
I get a 404 for the binaries. It's a INFRA thing I guess, because my
personal apache user page is also down/gone. :-(

On Tue, Mar 1, 2016 at 10:42 PM, Robert Metzger  wrote:
> Dear Flink community,
>
> Please vote on releasing the following candidate as Apache Flink version 1.0
> .0.
>
> This is the fourth RC.
> Here is a document to report on the testing and release verification:
> https://docs.google.com/document/d/1hoQ5k4WQteNj2OoPwpQPD4ZVHrCwM1pTlUVww8ld7oY/edit#heading=h.2v6zy51pgj33
>
>
> The commit to be voted on (
> http://git-wip-us.apache.org/repos/asf/flink/commit/1b0a8c4e)
> 1b0a8c4e9d5df35a7dea9cdd6d2c6e35489bfefa
>
> Branch:
> release-1.0.0-rc4 (see
> https://git1-us-west.apache.org/repos/asf/flink/repo?p=flink.git;a=shortlog;h=refs/heads/release-1.0.0-rc4
> )
>
> The release artifacts to be voted on can be found at:
> http://people.apache.org/~rmetzger/flink-1.0.0-rc4/
>
> The release artifacts are signed with the key with fingerprint D9839159:
> http://www.apache.org/dist/flink/KEYS
>
> The staging repository for this release can be found at:
> https://repository.apache.org/content/repositories/orgapacheflink-1066
>
> -
>
> The vote is open until Friday and passes if a majority of at least three
> +1 PMC votes are cast.
>
> The vote ends on Friday, March 4, 23:00 CET.
>
> [ ] +1 Release this package as Apache Flink 1.0.0
> [ ] -1 Do not release this package because ...