That is totally understandable. However, upgrading to a new version of
Flink is also a big change that could require additional changes which are
out of scope for a Beam minor release.

My advice, if you want to use the latest version but prevent changes coming
in constantly, you could use a fixed snapshot release. For example:

<dependency>
   <groupId>org.apache.beam</groupId>
   <artifactId>beam-runners-flink_2.10</artifactId>
   <version>0.3.0-incubating-20160920.071715-50</version>
</dependency>


You can derive this version umber from the snapshot repository: If you look
at the snapshot repository:
https://repository.apache.org/content/repositories/snapshots/org/apache/beam/beam-runners-flink_2.10/0.3.0-incubating-SNAPSHOT/maven-metadata.xml

It would be worthwhile to discuss how we handle version upgrades of
backends in the Beam release cycle. Unfortunately, for the Runner to be
compatible across Flink versions, we need a bit more than just API
stability because some parts integrate also with the Flink runtime. I can
see this becoming an issue once more runners are part of Beam.

Best,
Max

On Sun, Sep 18, 2016 at 11:04 PM, Chawla,Sumit <sumitkcha...@gmail.com>
wrote:

> Hi Max
>
> Thanks for the information. I agree with you that 0.3.0 is the way ahead,
> but i am hesitant to use 0.3.0-SNAPSHOT due to its changing nature.
>
> Regards
> Sumit Chawla
>
>
> On Fri, Sep 16, 2016 at 5:51 AM, Maximilian Michels <m...@apache.org>
> wrote:
>
>> Hi Sumit,
>>
>> Thanks for the PR. Your changes looks good. I think there are
>> currently no plans for a minor release 0.2.1-incubating. A lot of
>> issues were fixed on the latest master which should give you a better
>> experience than the 0.2.0-incubating release.
>>
>> These are the current issues which will be fixed in 0.3.0-incubating:
>> https://issues.apache.org/jira/browse/BEAM-102?jql=project%2
>> 0%3D%20BEAM%20AND%20fixVersion%20%3D%200.3.0-incubating%20AN
>> D%20component%20%3D%20runner-flink
>>
>> 1. Most notably, it fixes an issue with comparing keys unencoded in
>> streaming mode which was an issue if you had not implemented
>> equals/hashCode for your objects.
>>
>> 2. Further, we support side inputs in streaming now. In the course, we
>> have also unified execution paths in streaming mode.
>>
>> 3. There was an issue with checkpointed sources that has been resolved.
>>
>> If you could try out the latest version, that would be great. If not,
>> we can probably merge your PR and think about a minor release.
>>
>> Best,
>> Max
>>
>> On Fri, Sep 16, 2016 at 6:10 AM, Chawla,Sumit <sumitkcha...@gmail.com>
>> wrote:
>> > Hi Max
>> >
>> > I have opened a PR - https://github.com/apache/incubator-beam/pull/963
>> for
>> > adding support of Flink 1.1.2 in Beam 0.2.0 release.
>> >
>> > Regards
>> > Sumit Chawla
>> >
>> >
>> > On Wed, Sep 14, 2016 at 1:32 PM, Chawla,Sumit <sumitkcha...@gmail.com>
>> > wrote:
>> >
>> >> Hi Max
>> >>
>> >> I was able to compile 0.2.0 with Flink 1.1.0 with small modification,
>> and
>> >> run a simple pipeline.
>> >>
>> >>    @Override
>> >> -  public void restoreState(StreamTaskState taskState, long
>> >> recoveryTimestamp) throws Exception {
>> >> -    super.restoreState(taskState, recoveryTimestamp);
>> >> +  public void restoreState(StreamTaskState taskState) throws
>> Exception {
>> >> +    super.restoreState(taskState);
>> >>
>> >>
>> >> Can i get a sense of the changes that have happened in 0.3.0 for
>> Flink?  I
>> >> observed some classes completely reworked.  It will be crucial for me
>> to
>> >> understand the scope of change and impact before making a move to 0.3.0
>> >>
>> >>
>> >>
>> >> Regards
>> >> Sumit Chawla
>> >>
>> >>
>> >> On Wed, Sep 14, 2016 at 3:03 AM, Maximilian Michels <m...@apache.org>
>> >> wrote:
>> >>
>> >>> We support Flink 1.1.2 on the latest snapshot version
>> >>> 0.3.0-incubating-SNAPSHOT. Would it be possible for you to work with
>> >>> this version?
>> >>>
>> >>> On Tue, Sep 13, 2016 at 11:55 PM, Chawla,Sumit <
>> sumitkcha...@gmail.com>
>> >>> wrote:
>> >>> > When trying to use Beam 0.2.0 with Flink 1.1.0 jar, i am seeing
>> >>> following
>> >>> > error:
>> >>> >
>> >>> > java.lang.NoSuchMethodError:
>> >>> > org.apache.flink.streaming.api.operators.StreamingRuntimeCon
>> >>> text.registerTimer(JLorg/apache/flink/streaming/
>> >>> runtime/operators/Triggerable;)V
>> >>> >         at org.apache.beam.runners.flink.
>> translation.wrappers.streaming
>> >>> .io.UnboundedSourceWrapper.setNextWatermarkTimer(UnboundedSo
>> >>> urceWrapper.java:381)
>> >>> >         at org.apache.beam.runners.flink.
>> translation.wrappers.streaming
>> >>> .io.UnboundedSourceWrapper.run(UnboundedSourceWrapper.java:233)
>> >>> >         at org.apache.flink.streaming.api
>> .operators.StreamSource.run(
>> >>> StreamSource.java:80)
>> >>> >         at org.apache.flink.streaming.api
>> .operators.StreamSource.run(
>> >>> StreamSource.java:53)
>> >>> >         at org.apache.flink.streaming.run
>> time.tasks.SourceStreamTask.
>> >>> run(SourceStreamTask.java:56)
>> >>> >         at org.apache.flink.streaming.run
>> time.tasks.StreamTask.invoke(
>> >>> StreamTask.java:266)
>> >>> >         at org.apache.flink.runtime.taskm
>> anager.Task.run(Task.java:584)
>> >>> >         at java.lang.Thread.run(Thread.java:745)
>> >>> >
>> >>> >
>> >>> > Regards
>> >>> > Sumit Chawla
>> >>> >
>> >>> >
>> >>> > On Tue, Sep 13, 2016 at 2:20 PM, Chawla,Sumit <
>> sumitkcha...@gmail.com>
>> >>> > wrote:
>> >>> >
>> >>> >> Hi All
>> >>> >>
>> >>> >> The release-0.2.0-incubating supports Flink 1.0.3. With Flink 1.1.0
>> >>> out,
>> >>> >> is there a plan to support it with any 0.2.0 patch? I tried
>> compiling
>> >>> 0.2.0
>> >>> >> with Flink 1.1.0,
>> >>> >> and got couple of compliation errors in
>> FlinkGroupAlsoByWindowWrapper.
>> >>> java.
>> >>> >> Going back to master i see lots of change in Flink translation
>> >>> wrappers,
>> >>> >> and
>> >>> >> FlinkGroupAlsoByWindowWrapper.java being removed.
>> >>> >>
>> >>> >> Just want to get a sense of things here, on what would it take to
>> >>> support Flink
>> >>> >> 1.1.0 with release-0.2.0. Would appreciate views of people who are
>> >>> already
>> >>> >> working on upgrading it to Flink 1.1.0
>> >>> >>
>> >>> >> Regards
>> >>> >> Sumit Chawla
>> >>> >>
>> >>> >>
>> >>>
>> >>
>> >>
>>
>
>

Reply via email to