xintongsong commented on code in PR #23456: URL: https://github.com/apache/flink/pull/23456#discussion_r1336633757
########## flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java: ########## @@ -526,8 +526,14 @@ public DataStream<T> global() { * in the set time, the stream terminates. * * @return The iterative data stream created. + * @deprecated This method is deprecated since Flink 1.19. The users are recommended to use + * Iteration API in Flink ML instead. Review Comment: We should not recommend users to use Iteration API in Flink ML instead. It doesn't make sense that a user who doesn't need any ML algorithm would have to use Flink ML only to get access to the Iteration API. I'd suggest to the following: > The only known use case of this Iteration API comes from Flink ML, which already has its own implementation of iteration and no longer uses this API. If there's any use cases other than Flink ML that needs iteration support, please reach out to d...@flink.apache.org and we can consider making the Flink ML iteration implementation a separate common library. ########## flink-examples/flink-examples-streaming/pom.xml: ########## @@ -137,6 +137,11 @@ under the License. <configuration> <compilerArgument>-Xlint:deprecation</compilerArgument> <failOnWarning>true</failOnWarning> + <excludes> + <exclude> + org/apache/flink/streaming/examples/iteration/IterateExample.java Review Comment: We should explain that this example is temporarily preserved only for testing purpose. -- 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-unsubscr...@flink.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org