You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Aris Koliopoulos (Jira)" <ji...@apache.org> on 2020/07/05 14:21:00 UTC

[jira] [Commented] (FLINK-13414) Add support for Scala 2.13

    [ https://issues.apache.org/jira/browse/FLINK-13414?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17151590#comment-17151590 ] 

Aris Koliopoulos commented on FLINK-13414:
------------------------------------------

I had a go at it using two different ways:

*Standalone API in 2.13*

I tried to wrap flink-streaming-java_2.12 in a standalone separate Scala API in 2.13. I knew Akka would be the tricky part. There is binary compatibility between 2.12 and 2.13 and assuming classpath-related ClassDefNotFound issues can be resolved, this may work. I essentially tried to trick Flink into using Akka 2.5.23 (instead of 2.5.21) which has a 2.13 build. That didn't work due to shading Akka. The whole thing compiled but AkkaUtils has a number of references to deprecated collection APIs. I wonder If there is a way to shade Scala as well as Akka. In general, this feels like an overall bad idea. Even if we can work around Akka related issues, I would still very nervous about serializing/deseriazing 2.13 case classes with serialiazers compiled in 2.12.

Note: I had to copy the ClosureCleaner over and at least this part compiled without issues. 

*Flink in 2.13*

Akka only needs a minor version bump to 2.5.23 and I haven't seen any compilation issues so far. Chill needs a major bump from 0.7.6 to 0.9.5. A few APIs have changed which means org/apache/flink/runtime/types  needs to have two Scala folders, one for 2.12- and one for 2.13+. The changes are minor. I wouldn't do that though, I would update Chill to 0.9.5 first.

As @aljoscha predicted, due to the removal of `CanBuildFrom` the `TypeAnalyzer` needs some changes. In `TraversableSerializer` I had to replace `CanBuildFrom` with `Factory` from the new compatibility library.

There were a ton compiler errors due to a) Seq defaulting to immutable.Seq and b) mutable.MutableList being removed. Defaulting to immutable.Seq and replacing MutableList with ListBuffer did the trick at compile time. I haven't checked whether the new code compiles in 2.11/2.12 though.

After a few hours I got all modules up to flink-examples-batch to compile. I expect flink-table to need significant work;  the rest should be straightforward.

As far as tests are concerned, I ran all modules up to flink-runtime and I only encountered one failure in KryoSerializerRegistrationsTest.testDefaultKryoRegisteredClassesDidNotChange.

I suspect the bulk of the work will be around ensuring flink-scala tests run and pass for all supported Scala versions. 

I intend to spend a bit more time on this.

> Add support for Scala 2.13
> --------------------------
>
>                 Key: FLINK-13414
>                 URL: https://issues.apache.org/jira/browse/FLINK-13414
>             Project: Flink
>          Issue Type: New Feature
>          Components: API / Scala
>            Reporter: Chaoran Yu
>            Priority: Major
>




--
This message was sent by Atlassian Jira
(v8.3.4#803005)