You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2021/10/28 11:50:04 UTC

[GitHub] [flink] zentol opened a new pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

zentol opened a new pull request #17593:
URL: https://github.com/apache/flink/pull/17593


   This PR adds an e2e test for the Scala-free job execution.
   
   It covers cases where the flink-scala jar was removed from lib/ and:
   a) a Java job is run
   b) a Java job is run with a custom KryoSerializer (for completeness sake because we touched Kryo)
   c) Scala 2.11 is manually added to lib/, after which Scala job is run that uses the Java APIs.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25601",
       "triggerID" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579) 
   * 4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25601) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-960684706


   Did you build `flink-dist` recently (as in, since we package Scala separately)?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742751165



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
##########
@@ -223,6 +221,22 @@ public void submitSQLJob(SQLJobSubmission job, Duration timeout) throws IOExcept
                 .runBlocking(timeout);
     }
 
+    public void performJarAddition(JarAddition addition) throws IOException {
+        final Path target = mapJarLocationToPath(addition.getTarget());
+        final Path sourceJar = addition.getJar();
+
+        final String jarNameWithoutExtension =
+                FilenameUtils.removeExtension(sourceJar.getFileName().toString());

Review comment:
       because creating directories with file extensions is a bit weird isn't it




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742816924



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})
+@RunWith(Parameterized.class)

Review comment:
       Please don't refer to company-internal communication in reviews.
   
   Generally we do want to write new tests in jUnit 5. But until _existing_ utilities have been ported, this is sometimes just not possible, as is the case here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742752655



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml
##########
@@ -0,0 +1,227 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+		 xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<parent>
+		<artifactId>flink-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.15-SNAPSHOT</version>
+	</parent>
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-end-to-end-tests-scala</artifactId>
+	<name>Flink : E2E Tests : Scala</name>
+
+	<properties>
+		<!-- Use an old version that is not actively supported by Flink -->

Review comment:
       Overall, yes.
   I initially tried Scala 3 since that is a good way out afaik, but couldn't manage to set it up maven-wise.
   I did not want to use 2.13 because it may be supported soon.
   2.10 didn't work because of some plugin stuff, so I opted for 2.11.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "DELETED",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "status" : "SUCCESS",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25601",
       "triggerID" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9 Azure: [SUCCESS](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25601) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742703871



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})
+@RunWith(Parameterized.class)

Review comment:
       Here I'm not 100% sure whether it applies, but didn't we want to switch to junit5 for new tests?

##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
##########
@@ -223,6 +221,22 @@ public void submitSQLJob(SQLJobSubmission job, Duration timeout) throws IOExcept
                 .runBlocking(timeout);
     }
 
+    public void performJarAddition(JarAddition addition) throws IOException {
+        final Path target = mapJarLocationToPath(addition.getTarget());
+        final Path sourceJar = addition.getJar();
+
+        final String jarNameWithoutExtension =
+                FilenameUtils.removeExtension(sourceJar.getFileName().toString());

Review comment:
       Why do we remove the file extension here? 🤔 

##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/pom.xml
##########
@@ -0,0 +1,227 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+Licensed to the Apache Software Foundation (ASF) under one
+or more contributor license agreements.  See the NOTICE file
+distributed with this work for additional information
+regarding copyright ownership.  The ASF licenses this file
+to you under the Apache License, Version 2.0 (the
+"License"); you may not use this file except in compliance
+with the License.  You may obtain a copy of the License at
+  http://www.apache.org/licenses/LICENSE-2.0
+Unless required by applicable law or agreed to in writing,
+software distributed under the License is distributed on an
+"AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+KIND, either express or implied.  See the License for the
+specific language governing permissions and limitations
+under the License.
+-->
+<project xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd"
+		 xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance">
+
+	<parent>
+		<artifactId>flink-end-to-end-tests</artifactId>
+		<groupId>org.apache.flink</groupId>
+		<version>1.15-SNAPSHOT</version>
+	</parent>
+
+	<modelVersion>4.0.0</modelVersion>
+
+	<artifactId>flink-end-to-end-tests-scala</artifactId>
+	<name>Flink : E2E Tests : Scala</name>
+
+	<properties>
+		<!-- Use an old version that is not actively supported by Flink -->

Review comment:
       Could we also add a bit more of motivation. I guess, we go for an older version to avoid accidentally supporting it in the future?

##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})

Review comment:
       Out of curiosity: That's just some label which we didn't rename after migrating to AzureCI, isn't it?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol merged pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol merged pull request #17593:
URL: https://github.com/apache/flink/pull/17593


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742769282



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})
+@RunWith(Parameterized.class)

Review comment:
       I just remembered [this Slack thread](https://dataartisans.slack.com/archives/C1APHQMAT/p1634726247339700). Or were they referring to unit tests only for that one?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-960745548


   > Did you build flink-dist recently (as in, since we package Scala separately)?
   
   Yes, that fixed the test run 👍 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742968681



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/main/scala/org/apache/flink/tests/scala/ScalaJob.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.tests.scala
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+
+import scala.runtime.BoxesRunTime
+
+/**
+ * A Scala job that can only run with Scala 2.11.
+ *
+ * <p>This job also acts as a stand-on for Java jobs using some Scala library.
+ */
+object ScalaJob {
+  def main(args: Array[String]): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment()
+
+    // we want to go through serialization to check for kryo issues
+    env.disableOperatorChaining()
+
+    env.fromElements(new NonPojo()).map(new MapFunction[NonPojo, NonPojo] {
+      override def map(value: NonPojo): NonPojo = {
+        // use some method that was removed in 2.12+
+        BoxesRunTime.hashFromNumber(value.getSomeInt)

Review comment:
       Thanks for clarification. 👍 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol merged pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol merged pull request #17593:
URL: https://github.com/apache/flink/pull/17593


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol merged pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol merged pull request #17593:
URL: https://github.com/apache/flink/pull/17593


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742750729



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})
+@RunWith(Parameterized.class)

Review comment:
       Doesn't make sense to switch for this test when the entire e2e framework is build in junit4.

##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})

Review comment:
       yes




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "FAILURE",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     }, {
       "hash" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 Azure: [FAILURE](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579) 
   * 4ca2a2eeaf401789da55b5a15dd6a0e86306cfd9 UNKNOWN
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742757179



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-common/src/main/java/org/apache/flink/tests/util/flink/FlinkDistribution.java
##########
@@ -223,6 +221,22 @@ public void submitSQLJob(SQLJobSubmission job, Duration timeout) throws IOExcept
                 .runBlocking(timeout);
     }
 
+    public void performJarAddition(JarAddition addition) throws IOException {
+        final Path target = mapJarLocationToPath(addition.getTarget());
+        final Path sourceJar = addition.getJar();
+
+        final String jarNameWithoutExtension =
+                FilenameUtils.removeExtension(sourceJar.getFileName().toString());

Review comment:
       oh, I misinterpreted the `resolve` method chain. 🤦 makes sense




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742769282



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/test/java/org/apache/flink/tests/scala/ScalaFreeITCase.java
##########
@@ -0,0 +1,130 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.flink.tests.scala;
+
+import org.apache.flink.tests.util.TestUtils;
+import org.apache.flink.tests.util.categories.TravisGroup1;
+import org.apache.flink.tests.util.flink.ClusterController;
+import org.apache.flink.tests.util.flink.FlinkResource;
+import org.apache.flink.tests.util.flink.FlinkResourceSetup;
+import org.apache.flink.tests.util.flink.JarLocation;
+import org.apache.flink.tests.util.flink.JobSubmission;
+import org.apache.flink.testutils.executor.TestExecutorResource;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.file.Path;
+import java.time.Duration;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.function.Consumer;
+
+/**
+ * Tests that Flink does not require Scala for jobs that do not use the Scala APIs. This covers both
+ * pure Java jobs, and Scala jobs that use the Java APIs exclusively with Scala types.
+ */
+@Category(value = {TravisGroup1.class})
+@RunWith(Parameterized.class)

Review comment:
       [...]




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742785432



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/main/scala/org/apache/flink/tests/scala/ScalaJob.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.tests.scala
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+
+import scala.runtime.BoxesRunTime
+
+/**
+ * A Scala job that can only run with Scala 2.11.
+ *
+ * <p>This job also acts as a stand-on for Java jobs using some Scala library.
+ */
+object ScalaJob {
+  def main(args: Array[String]): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment()
+
+    // we want to go through serialization to check for kryo issues
+    env.disableOperatorChaining()
+
+    env.fromElements(new NonPojo()).map(new MapFunction[NonPojo, NonPojo] {
+      override def map(value: NonPojo): NonPojo = {
+        // use some method that was removed in 2.12+
+        BoxesRunTime.hashFromNumber(value.getSomeInt)

Review comment:
       Where can I see that `BoxesRunTime.hashFromNumber` was removed? I tried to verify for it through the [scala docs for 2.11.12](https://www.scala-lang.org/api/2.11.12) but couldn't find `BoxesRunTime`. 🤔 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot edited a comment on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot edited a comment on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953774184


   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "status" : "PENDING",
       "url" : "https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579",
       "triggerID" : "fb9bbc323b18e784b67dbe4691b3c7edbfbafb64",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 Azure: [PENDING](https://dev.azure.com/apache-flink/98463496-1af2-4620-8eab-a2ecc1a2e6fe/_build/results?buildId=25579) 
   
   <details>
   <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot run azure` re-run the last Azure build
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] flinkbot commented on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
flinkbot commented on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-953772007


   Thanks a lot for your contribution to the Apache Flink project. I'm the @flinkbot. I help the community
   to review your pull request. We will use this comment to track the progress of the review.
   
   
   ## Automated Checks
   Last check on commit fb9bbc323b18e784b67dbe4691b3c7edbfbafb64 (Thu Oct 28 11:52:43 UTC 2021)
   
   **Warnings:**
    * **2 pom.xml files were touched**: Check for build and licensing issues.
    * No documentation files were touched! Remember to keep the Flink docs up to date!
   
   
   <sub>Mention the bot in a comment to re-run the automated checks.</sub>
   ## Review Progress
   
   * ❓ 1. The [description] looks good.
   * ❓ 2. There is [consensus] that the contribution should go into to Flink.
   * ❓ 3. Needs [attention] from.
   * ❓ 4. The change fits into the overall [architecture].
   * ❓ 5. Overall code [quality] is good.
   
   Please see the [Pull Request Review Guide](https://flink.apache.org/contributing/reviewing-prs.html) for a full explanation of the review process.<details>
    The Bot is tracking the review progress through labels. Labels are applied according to the order of the review items. For consensus, approval by a Flink committer of PMC member is required <summary>Bot commands</summary>
     The @flinkbot bot supports the following commands:
   
    - `@flinkbot approve description` to approve one or more aspects (aspects: `description`, `consensus`, `architecture` and `quality`)
    - `@flinkbot approve all` to approve all aspects
    - `@flinkbot approve-until architecture` to approve everything until `architecture`
    - `@flinkbot attention @username1 [@username2 ..]` to require somebody's attention
    - `@flinkbot disapprove architecture` to remove an approval you gave earlier
   </details>


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] XComp commented on pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
XComp commented on pull request #17593:
URL: https://github.com/apache/flink/pull/17593#issuecomment-960745548


   > Did you build flink-dist recently (as in, since we package Scala separately)?
   Yes, that fixed the test run 👍 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [flink] zentol commented on a change in pull request #17593: [FLINK-24603][e2e] Add test for Scala-free Flink

Posted by GitBox <gi...@apache.org>.
zentol commented on a change in pull request #17593:
URL: https://github.com/apache/flink/pull/17593#discussion_r742809233



##########
File path: flink-end-to-end-tests/flink-end-to-end-tests-scala/src/main/scala/org/apache/flink/tests/scala/ScalaJob.scala
##########
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.tests.scala
+
+import org.apache.flink.api.common.functions.MapFunction
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment
+
+import scala.runtime.BoxesRunTime
+
+/**
+ * A Scala job that can only run with Scala 2.11.
+ *
+ * <p>This job also acts as a stand-on for Java jobs using some Scala library.
+ */
+object ScalaJob {
+  def main(args: Array[String]): Unit = {
+    val env = StreamExecutionEnvironment.getExecutionEnvironment()
+
+    // we want to go through serialization to check for kryo issues
+    env.disableOperatorChaining()
+
+    env.fromElements(new NonPojo()).map(new MapFunction[NonPojo, NonPojo] {
+      override def map(value: NonPojo): NonPojo = {
+        // use some method that was removed in 2.12+
+        BoxesRunTime.hashFromNumber(value.getSomeInt)

Review comment:
       Initially I looked at https://abi-laboratory.pro/index.php?view=compat_report&lang=java&l=scala&v1=2.11.12&v2=2.12.0&obj=5d4cf&kind=bin#Removed.
   
   Then I verified it in the Scala sources. [2.11](https://github.com/scala/scala/blob/2.11.x/src/library/scala/runtime/BoxesRunTime.java) vs [2.12](https://github.com/scala/scala/blob/2.12.x/src/library/scala/runtime/BoxesRunTime.java)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscribe@flink.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org