Re: [DISCUSS] API breaking change in DataStream Windows

2016-08-09 Thread Jark
As an user, I don’t like “casting option”. Because people who need set parallelism after CoGroup will certainly fall into this issue. They will subconsciously think Flink does not support this feature. We can’t assume most users will read JavaDocs and document carefully. Maybe we can post this

Re: [ANNOUNCE] Flink 1.1.0 Released

2016-08-09 Thread Fabian Hueske
Thanks Ufuk and everybody who contributed to the release! Cheers, Fabian 2016-08-08 20:41 GMT+02:00 Henry Saputra : > Great work all. Great Thanks to Ufuk as RE :) > > On Monday, August 8, 2016, Stephan Ewen wrote: > > > Great work indeed, and big thanks, Ufuk! > > > > On Mon, Aug 8, 2016 at 6:

Flink 1.1.0 : Hadoop 1/2 compatibility mismatch

2016-08-09 Thread LINZ, Arnaud
Hello, I’ve switched to 1.1.0, but part of my code doesn’t work any longer. Despite the fact that I have no Hadoop 1 jar in my dependencies (2.7.1 clients & flink-hadoop-compatibility_2.10 1.1.0), I have a weird JobContext version mismatch error, that I was unable to understand. Code is a hive

Re: [ANNOUNCE] Flink 1.1.0 Released

2016-08-09 Thread Matthias J. Sax
Congrats! On 08/09/2016 09:26 AM, Fabian Hueske wrote: > Thanks Ufuk and everybody who contributed to the release! > > Cheers, Fabian > > 2016-08-08 20:41 GMT+02:00 Henry Saputra : > >> Great work all. Great Thanks to Ufuk as RE :) >> >> On Monday, August 8, 2016, Stephan Ewen wrote: >> >>> Gr

[jira] [Created] (FLINK-4337) Remove unnecessary Scala suffix from Hadoop1 artifact

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4337: --- Summary: Remove unnecessary Scala suffix from Hadoop1 artifact Key: FLINK-4337 URL: https://issues.apache.org/jira/browse/FLINK-4337 Project: Flink Issue Type:

[DISCUSS] Streaming connector contributions

2016-08-09 Thread Robert Metzger
Hi, I would like to start a discussion regarding the streaming connectors in Flink. Currently, there are 12 connectors in the main repository, 4 more are pending as pull requests (rethinkdb, kafka 0.10, ActiveMQ, HBase), and there were some more discussed on the mailing list / JIRA / pull requests

[jira] [Created] (FLINK-4338) Implement Slot Pool

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4338: --- Summary: Implement Slot Pool Key: FLINK-4338 URL: https://issues.apache.org/jira/browse/FLINK-4338 Project: Flink Issue Type: Sub-task Components: Cl

[jira] [Created] (FLINK-4339) Implement Slot Pool Core

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4339: --- Summary: Implement Slot Pool Core Key: FLINK-4339 URL: https://issues.apache.org/jira/browse/FLINK-4339 Project: Flink Issue Type: Sub-task Component

Re: [DISCUSS] Streaming connector contributions

2016-08-09 Thread Tzu-Li (Gordon) Tai
Hi Robert, Thank you for bringing the discussion to the mailing lists. #2 seems like a good option, if we can reach consensus with the Bahir community. However, should we also be considering “staging” (perhaps under “flink-contrib”?) a connector contribution until more committers can maintain it

Re: [DISCUSS] Streaming connector contributions

2016-08-09 Thread Robert Metzger
Hi Gordon, thank you for your response. I agree with your observation that some "staging" area is helpful to test how many contributors / users are interested in a connector. But I wonder if #1 or #2 can also serve as a staging area: As soon as we see that there is a lot of interest in a connector

Re: [DISCUSS] updating apache-flink homebrew formula

2016-08-09 Thread Robert Metzger
Hi, Thanks a lot for maintaining the flink homebrew formula. I think its a good idea to update to Hadoop 2.7 and Scala 2.11. On Tue, Aug 9, 2016 at 12:59 AM, Wright, Eron wrote: > Hello, > > With the release of 1.1, I’m happy to update the apache-flink homebrew > package accordingly. Quick qu

Re: [DISCUSS] FLIP-5 Only send data to each taskmanager once for broadcasts

2016-08-09 Thread Till Rohrmann
Hi Felix, I'm not sure whether PipelinedSubpartition should be readable more than once because then it would effectively mean that we materialize the elements of the pipelined subpartition for stragglers. Therefore, I think that we should make blocking intermediate results readable more than once.

Re: [DISCUSS] Streaming connector contributions

2016-08-09 Thread Tzu-Li (Gordon) Tai
Good point. Discussion for each new connector is also an overhead we should avoid. IMHO, option #2 doesn’t seem like a reasonable staging area. Say we decide we’d like to move a connector from Bahir to Flink in the future, there’ll be two of the connector in separate Apache projects. Personally I

Re: Flink 1.1.0 : Hadoop 1/2 compatibility mismatch

2016-08-09 Thread Ufuk Celebi
As noted in the other thread, this is a problem with the Maven artifacts of 1.1.0 :-( I've added a warning to the release note and will start a emergency vote for 1.1.1 which only updates the Maven artifacts. On Tue, Aug 9, 2016 at 9:45 AM, LINZ, Arnaud wrote: > Hello, > > > > I’ve switched to 1.

Re: [DISCUSS] API breaking change in DataStream Windows

2016-08-09 Thread Greg Hogan
I agree that expecting users to cast is undesirable. Upon changing the API, why would we not mark the next release as 2.0? The same issue arose with Gabor's addition of hash-combine in the Scala DataSet API where DataSet was returned rather than a specialized Operator. The solution was to add an o

Re: [DISCUSS] updating apache-flink homebrew formula

2016-08-09 Thread Till Rohrmann
+1 for Hadoop 2.7 and Scala 2.11 On Tue, Aug 9, 2016 at 12:25 PM, Robert Metzger wrote: > Hi, > > Thanks a lot for maintaining the flink homebrew formula. > I think its a good idea to update to Hadoop 2.7 and Scala 2.11. > > On Tue, Aug 9, 2016 at 12:59 AM, Wright, Eron wrote: > > > Hello, > >

Re: [DISCUSS] updating apache-flink homebrew formula

2016-08-09 Thread Stephan Ewen
+1 for Hadoop 2.7 and Scala 2.11 Not sure if you saw some of the threads according to dependency issues in the 1.1.0 release. It may be worth to reference the soon-to-come 1.1.1 release instead. On Tue, Aug 9, 2016 at 2:36 PM, Till Rohrmann wrote: > +1 for Hadoop 2.7 and Scala 2.11 > > On Tue,

Re: [DISCUSS] Streaming connector contributions

2016-08-09 Thread Stephan Ewen
Thanks Robert for bringing this up. I think that "flink-contrib" will not really solve the problem, because the connectors would still have to be maintained by the core community, we would need to guarantee test stability. It will be to a large extend similar to adding them to "flink-streaming-con

[jira] [Created] (FLINK-4340) Remove RocksDB Semi-Async Checkpoint Mode

2016-08-09 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-4340: --- Summary: Remove RocksDB Semi-Async Checkpoint Mode Key: FLINK-4340 URL: https://issues.apache.org/jira/browse/FLINK-4340 Project: Flink Issue Type: Imp

Re: [DISCUSS] Streaming connector contributions

2016-08-09 Thread Till Rohrmann
I agree with Stephan that the main problem is maintenance overhead for the Flink community. If we could maintain all connectors ourselves then there would not be an immediate need to out source the connectors. Thus, the solution should reduce the workload for the core project. Personally, I would

Re: [DISCUSS] FLIP-5 Only send data to each taskmanager once for broadcasts

2016-08-09 Thread Felix Neutatz
Hi Till, thanks for the fast answer. I also think this should be the way to go. So should I open a new jira "Make blocking SpillableSubpartition able to be read multiple times". Moreover should I mark this jira and FLINK-1713 as blocking for the b

Re: [DISCUSS] API breaking change in DataStream Windows

2016-08-09 Thread Till Rohrmann
That is a tough call but I'm personally leaning slightly towards not breaking the API and adding a note for the casting workaround. My main concern is where do we set the limit for future API breaking issues? How critical does an issue has to be to be allowed to break the API? Currently, we have 1

Re: [DISCUSS] FLIP-5 Only send data to each taskmanager once for broadcasts

2016-08-09 Thread Till Rohrmann
Hi Felix, if we cannot work around the problem with blocking intermediate results in iterations, then we have to make FLINK-1713 a blocker for this new issue. But maybe you can also keep the current broadcasting mechanism to be used within iterations only. Then we can address the iteration problem

Re: [DISCUSS] FLIP-5 Only send data to each taskmanager once for broadcasts

2016-08-09 Thread Stephan Ewen
I agree with Till. Changing the basic data exchange mechanism would screw up many other ongoing efforts, like more incremental recovery. It seems to make this properly applicable, we need to first un-specialize the iterations. (1) Allow for "versioned" intermediate results, i.e., result-x-superst

Re: [DISCUSS] FLIP-6 - Flink Deployment and Process Model - Standalone, Yarn, Mesos, Kubernetes, etc.

2016-08-09 Thread Stephan Ewen
@Kurt You raise some good points. These are tricky issues indeed. Here are some thoughts: (1) I think the resources required for a function can only be decided by the user (at least in a first version). If I recall correctly think Blink used annotations in Yarn to the user code to define how man

Re: [DISCUSS] FLIP-6 - Flink Deployment and Process Model - Standalone, Yarn, Mesos, Kubernetes, etc.

2016-08-09 Thread Stephan Ewen
Hi Eron! As per our separate discussion, the main thing here is a confusion about what a session is. I would keep the initial API of the dispatcher to just "launchJob(JobGraph, artifacts)". It would simply start individual jobs in the cluster. If we want to make the dispatcher "resource session"

N-ary stream operators - status

2016-08-09 Thread Gábor Gévay
Hello, There is this Google Doc about adding n-ary stream operators to Flink: https://docs.google.com/document/d/1ZFzL_0xGuUEnBsFyEiHwWcmCcjhd9ArWsmhrgnt05RI/edit I would like to ask what are the plans for when will this feature be available? Best, Gábor

[VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Ufuk Celebi
Dear Flink community, Please vote on releasing the following candidate as Apache Flink version 1.1.1. *Important*: I would like to reduce the voting time to 24 hours (with a majority of at least three +1 PMC votes as usual). We discovered that the Maven artifacts published with version 1.1.0 have

Re: Flink 1.1.0 : Hadoop 1/2 compatibility mismatch

2016-08-09 Thread Ufuk Celebi
I've started a vote for 1.1.1 containing hopefully fixed artifacts. If you have any spare time, would you mind checking whether it fixes your problem? The artifacts are here: http://home.apache.org/~uce/flink-1.1.1-rc1/ You would have to add the following repository to your Maven project and upda

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Ufuk Celebi
PS: Let me add that no changes other than the release 1.1.1 commit have been added on top of what we had in 1.1.0. On Tue, Aug 9, 2016 at 6:27 PM, Ufuk Celebi wrote: > Dear Flink community, > > Please vote on releasing the following candidate as Apache Flink version > 1.1.1. > > *Important*: I w

TaskManager Transfer buffers

2016-08-09 Thread David Herzog
Hi dear Support, I am currently working on a research project and have to understand how the transfer of buffers exactly works. https://cwiki.apache.org/confluence/display/FLINK/Data+exchange+between+tasks This page describes how a request for a ResultSubpartion works. I got to the part that *Part

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Henry Saputra
Yes, We can do this with reduced time for VOTE On Tuesday, August 9, 2016, Ufuk Celebi wrote: > PS: Let me add that no changes other than the release 1.1.1 commit > have been added on top of what we had in 1.1.0. > > On Tue, Aug 9, 2016 at 6:27 PM, Ufuk Celebi > > wrote: > > Dear Flink community

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Márton Balassi
Go ahead. +1, (binding). On Tue, Aug 9, 2016 at 6:53 PM, Henry Saputra wrote: > Yes, We can do this with reduced time for VOTE > > On Tuesday, August 9, 2016, Ufuk Celebi wrote: > > > PS: Let me add that no changes other than the release 1.1.1 commit > > have been added on top of what we had in

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Vasiliki Kalavri
On 9 August 2016 at 18:27, Ufuk Celebi wrote: > Dear Flink community, > > Please vote on releasing the following candidate as Apache Flink version > 1.1.1. > > *Important*: I would like to reduce the voting time to 24 hours (with > a majority of at least three +1 PMC votes as usual). We discovere

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Gyula Fóra
+1 from me, this is a very important fix. Gyula Vasiliki Kalavri ezt írta (időpont: 2016. aug. 9., K, 19:15): > On 9 August 2016 at 18:27, Ufuk Celebi wrote: > > > Dear Flink community, > > > > Please vote on releasing the following candidate as Apache Flink version > > 1.1.1. > > > > *Importa

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Henry Saputra
Official vote +1 (binding) On Tuesday, August 9, 2016, Gyula Fóra wrote: > +1 from me, this is a very important fix. > > Gyula > > Vasiliki Kalavri > ezt írta > (időpont: 2016. aug. > 9., K, 19:15): > > > On 9 August 2016 at 18:27, Ufuk Celebi > > wrote: > > > > > Dear Flink community, > > > > >

Re: [VOTE] Release Apache Flink 1.1.1 (RC1)

2016-08-09 Thread Stephan Ewen
+1 This is a crucial fix and the released sources are actually still the same, so reduced time should be okay. On Tue, Aug 9, 2016 at 8:24 PM, Henry Saputra wrote: > Official vote > +1 (binding) > > On Tuesday, August 9, 2016, Gyula Fóra wrote: > > > +1 from me, this is a very important fix. >

[jira] [Created] (FLINK-4341) Checkpoint state size grows unbounded when task parallelism not uniform

2016-08-09 Thread Scott Kidder (JIRA)
Scott Kidder created FLINK-4341: --- Summary: Checkpoint state size grows unbounded when task parallelism not uniform Key: FLINK-4341 URL: https://issues.apache.org/jira/browse/FLINK-4341 Project: Flink

[jira] [Created] (FLINK-4342) Fix dependencies of flink-connector-filesystem

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4342: --- Summary: Fix dependencies of flink-connector-filesystem Key: FLINK-4342 URL: https://issues.apache.org/jira/browse/FLINK-4342 Project: Flink Issue Type: Bug

Re: [DISCUSS] updating apache-flink homebrew formula

2016-08-09 Thread Wright, Eron
Will update the homebrew package to Flink 1.1.1 + Hadoop 2.7 + Scala 2.11. > On Aug 9, 2016, at 5:48 AM, Stephan Ewen wrote: > > +1 for Hadoop 2.7 and Scala 2.11 > > Not sure if you saw some of the threads according to dependency issues in > the 1.1.0 release. It may be worth to reference the s

[jira] [Created] (FLINK-4343) Implement new TaskManager

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4343: --- Summary: Implement new TaskManager Key: FLINK-4343 URL: https://issues.apache.org/jira/browse/FLINK-4343 Project: Flink Issue Type: New Feature Compo

[jira] [Created] (FLINK-4344) Implement new JobManager

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4344: --- Summary: Implement new JobManager Key: FLINK-4344 URL: https://issues.apache.org/jira/browse/FLINK-4344 Project: Flink Issue Type: New Feature Compon

[jira] [Created] (FLINK-4345) Implement new ResourceManager

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4345: --- Summary: Implement new ResourceManager Key: FLINK-4345 URL: https://issues.apache.org/jira/browse/FLINK-4345 Project: Flink Issue Type: New Feature C

[jira] [Created] (FLINK-4346) Implement basic RPC abstraction

2016-08-09 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-4346: --- Summary: Implement basic RPC abstraction Key: FLINK-4346 URL: https://issues.apache.org/jira/browse/FLINK-4346 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-4347) Implement SlotManager for new ResourceManager

2016-08-09 Thread Kurt Young (JIRA)
Kurt Young created FLINK-4347: - Summary: Implement SlotManager for new ResourceManager Key: FLINK-4347 URL: https://issues.apache.org/jira/browse/FLINK-4347 Project: Flink Issue Type: Sub-task