A master project about implementing Cypher on Apache Flink

2016-10-14 Thread Mengqi Yang
Hi all, Let me introduce me first. My name is Mengqi Yang. I was a master student in Eindhoven University of Technology. Recently I have finished my master project there (Vasia is also one of my supervisors, great thanks to her :) ). The name of this project is "a study of execution strategies

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Fabian Hueske
Yes, I'm also +1 for removing the methods at some point. For 1.2 we could go ahead and move the Hadoop-MR connectors into a separate module and mark the methods in ExecutionEnvironment as @deprecated. In 1.3 (or 2.0 whatever comes next) we could make the switch. 2016-10-14 10:40 GMT+02:00

Re: RE:[DISCUSS] FLIP-11: Table API Stream Aggregations

2016-10-14 Thread Timo Walther
Hi everyone, I think syntax in general is a question of taste, it will be hard to make everyone happy. On the one hand it would be great if Table API and SQL could look consistent, but on the other hand there are also some negative aspects: SQL is a language that has not been developed for

[DISCUSS] Deprecate Hadoop source method from (batch) ExecutionEnvironment

2016-10-14 Thread Fabian Hueske
Hi everybody, I would like to propose to deprecate the utility methods to read data with Hadoop InputFormats from the (batch) ExecutionEnvironment. The motivation for deprecating these methods is reduce Flink's dependency on Hadoop but rather have Hadoop as an optional dependency for users that

Re: [DISCUSS] Deprecate Hadoop source method from (batch) ExecutionEnvironment

2016-10-14 Thread Greg Hogan
+1 On Fri, Oct 14, 2016 at 5:29 AM, Fabian Hueske wrote: > Hi everybody, > > I would like to propose to deprecate the utility methods to read data with > Hadoop InputFormats from the (batch) ExecutionEnvironment. > > The motivation for deprecating these methods is reduce

Re: 答复: RE:[DISCUSS] FLIP-11: Table API Stream Aggregations

2016-10-14 Thread Sean Wang
Thanks for your quick reply, Fabian. I have a few minor comments: - Agree that we should consider GroupBy without window after the new SQL proposal is settled down. - For Java API, we can keep window() call, and put window alias into Groupby clause. This can be also applied to rowwindow case.

Re: [DISCUSS] Deprecate Hadoop source method from (batch) ExecutionEnvironment

2016-10-14 Thread Robert Metzger
+1 On Fri, Oct 14, 2016 at 12:04 PM, Stephan Ewen wrote: > +1 > > On Fri, Oct 14, 2016 at 11:54 AM, Greg Hogan wrote: > > > +1 > > > > On Fri, Oct 14, 2016 at 5:29 AM, Fabian Hueske > wrote: > > > > > Hi everybody, > > > > > > I would

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Stephan Ewen
@Greg I think that would be amazing. It does require a bit of cleanup, though. As far as I know, the Hadoop dependency is additionally used for some Kerberos utilities and for its S3 file system implementation. We would need to make the Kerberos part Hadoop independent and the FileSystem loading

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Stephan Ewen
@Fabian - Someone started with that in https://issues.apache.org/jira/browse/FLINK-4315 That could be changed to not remove the methods from the ExecutionEnvironment. On Fri, Oct 14, 2016 at 10:45 AM, Fabian Hueske wrote: > Yes, I'm also +1 for removing the methods at some

[jira] [Created] (FLINK-4829) Accumulators are not thread safe

2016-10-14 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-4829: Summary: Accumulators are not thread safe Key: FLINK-4829 URL: https://issues.apache.org/jira/browse/FLINK-4829 Project: Flink Issue Type: Bug

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Stephan Ewen
@Fabian Good point. For Flink 2.0, I would suggest to remove them from the Environment and add them to a Utility. The way it is now, it ties Flink very strongly to Hadoop. You are right, before we do that, there is no way to make a Hadoop independent distribution. On Fri, Oct 14, 2016 at 10:37

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Fabian Hueske
Thanks for the pointer. I'll start a separate discussion and push the PR forward if we come to an agreement. 2016-10-14 11:04 GMT+02:00 Stephan Ewen : > @Fabian - Someone started with that in > https://issues.apache.org/jira/browse/FLINK-4315 > That could be changed to not

Re: [DISCUSS] Drop Hadoop 1 support with Flink 1.2

2016-10-14 Thread Fabian Hueske
+1 for dropping Hadoop1 support. Regarding a binary release without Hadoop: What would we do about the readHadoopFile() and createHadoopInput() on the ExecutionEnvironment? These methods are declared as @PublicEvolving, so we did not commit to keep them. However that does not necessarily mean we

[jira] [Created] (FLINK-4828) execute stream job asynchronously

2016-10-14 Thread David Anderson (JIRA)
David Anderson created FLINK-4828: - Summary: execute stream job asynchronously Key: FLINK-4828 URL: https://issues.apache.org/jira/browse/FLINK-4828 Project: Flink Issue Type: New Feature

Re: [DISCUSS] Deprecate Hadoop source method from (batch) ExecutionEnvironment

2016-10-14 Thread Stephan Ewen
+1 On Fri, Oct 14, 2016 at 11:54 AM, Greg Hogan wrote: > +1 > > On Fri, Oct 14, 2016 at 5:29 AM, Fabian Hueske wrote: > > > Hi everybody, > > > > I would like to propose to deprecate the utility methods to read data > with > > Hadoop InputFormats from the

[jira] [Created] (FLINK-4831) IMplement a log4j metric reporter

2016-10-14 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-4831: --- Summary: IMplement a log4j metric reporter Key: FLINK-4831 URL: https://issues.apache.org/jira/browse/FLINK-4831 Project: Flink Issue Type:

[jira] [Created] (FLINK-4830) Show latency statistics in web interface

2016-10-14 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-4830: - Summary: Show latency statistics in web interface Key: FLINK-4830 URL: https://issues.apache.org/jira/browse/FLINK-4830 Project: Flink Issue Type:

[jira] [Created] (FLINK-4832) Count/Sum 0 elements

2016-10-14 Thread Timo Walther (JIRA)
Timo Walther created FLINK-4832: --- Summary: Count/Sum 0 elements Key: FLINK-4832 URL: https://issues.apache.org/jira/browse/FLINK-4832 Project: Flink Issue Type: Improvement

Re: Exception from in-progress implementation of Python API bulk iterations

2016-10-14 Thread Chesnay Schepler
In this branch: https://github.com/zentol/flink/tree/new-iterations you can find a more fine-grained fix for chaining with iterations. relevant commit: ac2305d9589a5c6ab9e94d04c870fba52716d695 On 13.10.2016 23:11, Chesnay Schepler wrote: The chaining code is definitely related, I also have a