Re: FLINK-9752 (s3 recoverable writer) not actually fixed in 1.6.2?

2018-10-31 Thread Kostas Kloudas
Hi Addison, This is definitely an error on my end, as the feature is going to be available from Flink 1.7 onwards. I forgot to correctly update the Flink version when closing the JIRA issue. I will update the release notes accordingly. Sorry for the miscommunication, Kostas > On Oct 31,

Re: FLINK-9752 (s3 recoverable writer) not actually fixed in 1.6.2?

2018-10-30 Thread Addison Higham
I have backported this at https://github.com/instructure/flink/tree/s3_recover_backport by cherry-picking all the relevant code, I am not sure how backports are usually done with Flink (if you squash and merge) but there were a few minor conflicts and involved quite a few changes from master.

Re: FLINK-9752 (s3 recoverable writer) not actually fixed in 1.6.2?

2018-10-30 Thread Mike Mintz
FWIW I also tried this on Flink 1.6.2 today and got the same error. This is my full stack trace: java.lang.UnsupportedOperationException: Recoverable writers on Hadoop are only supported for HDFS and for Hadoop version 2.7 or newer at

Re: FLINK-9752 (s3 recoverable writer) not actually fixed in 1.6.2?

2018-10-30 Thread Till Rohrmann
Hi Addison, I think the idea was to also backport this feature to 1.6 since we considered it a bug that S3 was not supported in 1.6. I've pulled in Kostas who worked on the S3 writer. @Klou did we intentionally not backport this feature? I think there should be nothing special about backporting

FLINK-9752 (s3 recoverable writer) not actually fixed in 1.6.2?

2018-10-30 Thread Addison Higham
Hi all, Been hitting my head against a wall for the last few hours. The release notes for 1.6.2 show https://issues.apache.org/jira/browse/FLINK-9752 as resolved in 1.6.2. I am trying to upgrade and switch some things to use the StreamingFileSink against s3. However, when doing so, I get the