Hi all!
We have a problem in the *DataStream API* around Windows for *CoGroup* and *Join*. These operations currently do not allow to set a parallelism, which is a pretty heavy problem. To fix it properly, we need to change the return types of the coGroup() and join() operations, which *breaks the binary compatibility* - it* retains source compatibility*, though. The pull request with the change is: https://github.com/apache/flink/pull/2305 There are very clumsy ways to work around this (custom casts in the user code or making the join() / coGroup() behave differently than the other operators) which we did not really think of as viable, because they would need to be changed again in the future once we pull the API straight (breaking even source compatibility then). *I would suggest to actually break the API* at that point (binary, not source) for *Flink 1.2* and add a big note in the release docs. An uncomfortable step, but the alternatives are quite bad, too. Have a look at what has been suggested in the pull request discussion and please let us know what you think about that so we can proceed. Greetings, Stephan |
Thank you for bringing this discussion to the mailing list.
I agree with Chesnay's comment on GitHub that we should consider the "casting option" as well. I don't think we should break the API if there is a (admittedly ugly) work-around for those who run into the problem. If we put the work-around into the JavaDocs and the DataStream API documentation, everybody who is seriously blocked by this should find it. For 2.0, we can break the API by changing the method signature. On Mon, Aug 8, 2016 at 4:11 PM, Stephan Ewen <[hidden email]> wrote: > Hi all! > > We have a problem in the *DataStream API* around Windows for *CoGroup* and > *Join*. > These operations currently do not allow to set a parallelism, which is a > pretty heavy problem. > > To fix it properly, we need to change the return types of the coGroup() and > join() operations, which *breaks the binary compatibility* - it* retains > source compatibility*, though. > > The pull request with the change is: > https://github.com/apache/flink/pull/2305 > > There are very clumsy ways to work around this (custom casts in the user > code or making the join() / coGroup() behave differently than the other > operators) which we did not really think of as viable, because they would > need to be changed again in the future once we pull the API straight > (breaking even source compatibility then). > > *I would suggest to actually break the API* at that point (binary, not > source) for *Flink 1.2* and add a big note in the release docs. An > uncomfortable step, but the alternatives are quite bad, too. > > Have a look at what has been suggested in the pull request discussion and > please let us know what you think about that so we can proceed. > > Greetings, > Stephan > |
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 mail to the user list, and listen whether most users can accept the “casting” work-around and which work-around they prefer. > 在 2016年8月8日,下午10:45,Robert Metzger <[hidden email]> 写道: > > Thank you for bringing this discussion to the mailing list. > > I agree with Chesnay's comment on GitHub that we should consider the > "casting option" as well. I don't think we should break the API if there is > a (admittedly ugly) work-around for those who run into the problem. > If we put the work-around into the JavaDocs and the DataStream API > documentation, everybody who is seriously blocked by this should find it. > > For 2.0, we can break the API by changing the method signature. > > > On Mon, Aug 8, 2016 at 4:11 PM, Stephan Ewen <[hidden email]> wrote: > >> Hi all! >> >> We have a problem in the *DataStream API* around Windows for *CoGroup* and >> *Join*. >> These operations currently do not allow to set a parallelism, which is a >> pretty heavy problem. >> >> To fix it properly, we need to change the return types of the coGroup() and >> join() operations, which *breaks the binary compatibility* - it* retains >> source compatibility*, though. >> >> The pull request with the change is: >> https://github.com/apache/flink/pull/2305 >> >> There are very clumsy ways to work around this (custom casts in the user >> code or making the join() / coGroup() behave differently than the other >> operators) which we did not really think of as viable, because they would >> need to be changed again in the future once we pull the API straight >> (breaking even source compatibility then). >> >> *I would suggest to actually break the API* at that point (binary, not >> source) for *Flink 1.2* and add a big note in the release docs. An >> uncomfortable step, but the alternatives are quite bad, too. >> >> Have a look at what has been suggested in the pull request discussion and >> please let us know what you think about that so we can proceed. >> >> Greetings, >> Stephan >> |
In reply to this post by Stephan Ewen
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 overloaded method. https://github.com/apache/flink/pull/1517/files#diff-b1fea8d5283d978e9ccbc202dad36b5fR322 On Mon, Aug 8, 2016 at 10:11 AM, Stephan Ewen <[hidden email]> wrote: > Hi all! > > We have a problem in the *DataStream API* around Windows for *CoGroup* and > *Join*. > These operations currently do not allow to set a parallelism, which is a > pretty heavy problem. > > To fix it properly, we need to change the return types of the coGroup() and > join() operations, which *breaks the binary compatibility* - it* retains > source compatibility*, though. > > The pull request with the change is: > https://github.com/apache/flink/pull/2305 > > There are very clumsy ways to work around this (custom casts in the user > code or making the join() / coGroup() behave differently than the other > operators) which we did not really think of as viable, because they would > need to be changed again in the future once we pull the API straight > (breaking even source compatibility then). > > *I would suggest to actually break the API* at that point (binary, not > source) for *Flink 1.2* and add a big note in the release docs. An > uncomfortable step, but the alternatives are quite bad, too. > > Have a look at what has been suggested in the pull request discussion and > please let us know what you think about that so we can proceed. > > Greetings, > Stephan > |
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 10 API breaking issues for Flink 2.0 [1]. Why not including one of them as well? I think that backwards compatibility (source as well as binary) is really important for many users and we have the duty to live up to our promises. Imho, if things are API breaking, then we should indeed bump the major version, as Greg suggested. [1] https://issues.apache.org/jira/browse/FLINK-3957 Cheers, Till On Tue, Aug 9, 2016 at 2:20 PM, Greg Hogan <[hidden email]> wrote: > 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 overloaded method. > > https://github.com/apache/flink/pull/1517/files#diff- > b1fea8d5283d978e9ccbc202dad36b5fR322 > > On Mon, Aug 8, 2016 at 10:11 AM, Stephan Ewen <[hidden email]> wrote: > > > Hi all! > > > > We have a problem in the *DataStream API* around Windows for *CoGroup* > and > > *Join*. > > These operations currently do not allow to set a parallelism, which is a > > pretty heavy problem. > > > > To fix it properly, we need to change the return types of the coGroup() > and > > join() operations, which *breaks the binary compatibility* - it* retains > > source compatibility*, though. > > > > The pull request with the change is: > > https://github.com/apache/flink/pull/2305 > > > > There are very clumsy ways to work around this (custom casts in the user > > code or making the join() / coGroup() behave differently than the other > > operators) which we did not really think of as viable, because they would > > need to be changed again in the future once we pull the API straight > > (breaking even source compatibility then). > > > > *I would suggest to actually break the API* at that point (binary, not > > source) for *Flink 1.2* and add a big note in the release docs. An > > uncomfortable step, but the alternatives are quite bad, too. > > > > Have a look at what has been suggested in the pull request discussion and > > please let us know what you think about that so we can proceed. > > > > Greetings, > > Stephan > > > |
I would like have a decision on this so that we can merge the pull request.
If we can not come up with a solution everybody agrees, and nobody rejects the VOTE, I'll start a VOTE thread in 24 hours. On Tue, Aug 9, 2016 at 3:57 PM, Till Rohrmann <[hidden email]> wrote: > 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 10 API breaking issues for Flink 2.0 [1]. Why not > including one of them as well? > > I think that backwards compatibility (source as well as binary) is really > important for many users and we have the duty to live up to our promises. > Imho, if things are API breaking, then we should indeed bump the major > version, as Greg suggested. > > [1] https://issues.apache.org/jira/browse/FLINK-3957 > > Cheers, > Till > > On Tue, Aug 9, 2016 at 2:20 PM, Greg Hogan <[hidden email]> wrote: > > > 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 overloaded method. > > > > https://github.com/apache/flink/pull/1517/files#diff- > > b1fea8d5283d978e9ccbc202dad36b5fR322 > > > > On Mon, Aug 8, 2016 at 10:11 AM, Stephan Ewen <[hidden email]> wrote: > > > > > Hi all! > > > > > > We have a problem in the *DataStream API* around Windows for *CoGroup* > > and > > > *Join*. > > > These operations currently do not allow to set a parallelism, which is > a > > > pretty heavy problem. > > > > > > To fix it properly, we need to change the return types of the coGroup() > > and > > > join() operations, which *breaks the binary compatibility* - it* > retains > > > source compatibility*, though. > > > > > > The pull request with the change is: > > > https://github.com/apache/flink/pull/2305 > > > > > > There are very clumsy ways to work around this (custom casts in the > user > > > code or making the join() / coGroup() behave differently than the other > > > operators) which we did not really think of as viable, because they > would > > > need to be changed again in the future once we pull the API straight > > > (breaking even source compatibility then). > > > > > > *I would suggest to actually break the API* at that point (binary, not > > > source) for *Flink 1.2* and add a big note in the release docs. An > > > uncomfortable step, but the alternatives are quite bad, too. > > > > > > Have a look at what has been suggested in the pull request discussion > and > > > please let us know what you think about that so we can proceed. > > > > > > Greetings, > > > Stephan > > > > > > |
I forgot to start the VOTE after 24 hours.
However, I checked again Apache's voting rules [1] and code changes require consensus. So a -1 vote by a PMC member effectively is a veto. I don't have the impression from the discussion that we have a clear majority for one approach, so a VOTE thread would quickly turn into another big discussion thread. Let me summarize the options at hand: #1 Break the DataStream API, by changing the return type of the apply() method #2 Keep the API and ask users to manually cast the return type of apply() to change the parallelism #3 Introduce a deprecated with() method that returns the correct type I prefer #2, because I think the should not break the API. We had some other API breaking discussions and we always found a solution. [1] http://www.apache.org/foundation/voting.html#votes-on-code-modification On Mon, Aug 15, 2016 at 12:21 PM, Robert Metzger <[hidden email]> wrote: > I would like have a decision on this so that we can merge the pull request. > > If we can not come up with a solution everybody agrees, and nobody rejects > the VOTE, I'll start a VOTE thread in 24 hours. > > > On Tue, Aug 9, 2016 at 3:57 PM, Till Rohrmann <[hidden email]> > wrote: > >> 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 10 API breaking issues for Flink 2.0 [1]. Why not >> including one of them as well? >> >> I think that backwards compatibility (source as well as binary) is really >> important for many users and we have the duty to live up to our promises. >> Imho, if things are API breaking, then we should indeed bump the major >> version, as Greg suggested. >> >> [1] https://issues.apache.org/jira/browse/FLINK-3957 >> >> Cheers, >> Till >> >> On Tue, Aug 9, 2016 at 2:20 PM, Greg Hogan <[hidden email]> wrote: >> >> > 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 overloaded method. >> > >> > https://github.com/apache/flink/pull/1517/files#diff- >> > b1fea8d5283d978e9ccbc202dad36b5fR322 >> > >> > On Mon, Aug 8, 2016 at 10:11 AM, Stephan Ewen <[hidden email]> wrote: >> > >> > > Hi all! >> > > >> > > We have a problem in the *DataStream API* around Windows for *CoGroup* >> > and >> > > *Join*. >> > > These operations currently do not allow to set a parallelism, which >> is a >> > > pretty heavy problem. >> > > >> > > To fix it properly, we need to change the return types of the >> coGroup() >> > and >> > > join() operations, which *breaks the binary compatibility* - it* >> retains >> > > source compatibility*, though. >> > > >> > > The pull request with the change is: >> > > https://github.com/apache/flink/pull/2305 >> > > >> > > There are very clumsy ways to work around this (custom casts in the >> user >> > > code or making the join() / coGroup() behave differently than the >> other >> > > operators) which we did not really think of as viable, because they >> would >> > > need to be changed again in the future once we pull the API straight >> > > (breaking even source compatibility then). >> > > >> > > *I would suggest to actually break the API* at that point (binary, not >> > > source) for *Flink 1.2* and add a big note in the release docs. An >> > > uncomfortable step, but the alternatives are quite bad, too. >> > > >> > > Have a look at what has been suggested in the pull request discussion >> and >> > > please let us know what you think about that so we can proceed. >> > > >> > > Greetings, >> > > Stephan >> > > >> > >> > > |
I think no vote is needed. The pull request is actually converging towards
a decision. On Thu, Aug 18, 2016 at 12:07 PM, Robert Metzger <[hidden email]> wrote: > I forgot to start the VOTE after 24 hours. > However, I checked again Apache's voting rules [1] and code changes require > consensus. So a -1 vote by a PMC member effectively is a veto. > I don't have the impression from the discussion that we have a clear > majority for one approach, so a VOTE thread would quickly turn into another > big discussion thread. > > Let me summarize the options at hand: > #1 Break the DataStream API, by changing the return type of the apply() > method > #2 Keep the API and ask users to manually cast the return type of apply() > to change the parallelism > #3 Introduce a deprecated with() method that returns the correct type > > > I prefer #2, because I think the should not break the API. We had some > other API breaking discussions and we always found a solution. > > > [1] http://www.apache.org/foundation/voting.html#votes- > on-code-modification > > On Mon, Aug 15, 2016 at 12:21 PM, Robert Metzger <[hidden email]> > wrote: > > > I would like have a decision on this so that we can merge the pull > request. > > > > If we can not come up with a solution everybody agrees, and nobody > rejects > > the VOTE, I'll start a VOTE thread in 24 hours. > > > > > > On Tue, Aug 9, 2016 at 3:57 PM, Till Rohrmann <[hidden email]> > > wrote: > > > >> 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 10 API breaking issues for Flink 2.0 [1]. Why > not > >> including one of them as well? > >> > >> I think that backwards compatibility (source as well as binary) is > really > >> important for many users and we have the duty to live up to our > promises. > >> Imho, if things are API breaking, then we should indeed bump the major > >> version, as Greg suggested. > >> > >> [1] https://issues.apache.org/jira/browse/FLINK-3957 > >> > >> Cheers, > >> Till > >> > >> On Tue, Aug 9, 2016 at 2:20 PM, Greg Hogan <[hidden email]> wrote: > >> > >> > 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 overloaded method. > >> > > >> > https://github.com/apache/flink/pull/1517/files#diff- > >> > b1fea8d5283d978e9ccbc202dad36b5fR322 > >> > > >> > On Mon, Aug 8, 2016 at 10:11 AM, Stephan Ewen <[hidden email]> > wrote: > >> > > >> > > Hi all! > >> > > > >> > > We have a problem in the *DataStream API* around Windows for > *CoGroup* > >> > and > >> > > *Join*. > >> > > These operations currently do not allow to set a parallelism, which > >> is a > >> > > pretty heavy problem. > >> > > > >> > > To fix it properly, we need to change the return types of the > >> coGroup() > >> > and > >> > > join() operations, which *breaks the binary compatibility* - it* > >> retains > >> > > source compatibility*, though. > >> > > > >> > > The pull request with the change is: > >> > > https://github.com/apache/flink/pull/2305 > >> > > > >> > > There are very clumsy ways to work around this (custom casts in the > >> user > >> > > code or making the join() / coGroup() behave differently than the > >> other > >> > > operators) which we did not really think of as viable, because they > >> would > >> > > need to be changed again in the future once we pull the API straight > >> > > (breaking even source compatibility then). > >> > > > >> > > *I would suggest to actually break the API* at that point (binary, > not > >> > > source) for *Flink 1.2* and add a big note in the release docs. An > >> > > uncomfortable step, but the alternatives are quite bad, too. > >> > > > >> > > Have a look at what has been suggested in the pull request > discussion > >> and > >> > > please let us know what you think about that so we can proceed. > >> > > > >> > > Greetings, > >> > > Stephan > >> > > > >> > > >> > > > > > |
Free forum by Nabble | Edit this page |