kl0u commented on a change in pull request #13509:
URL: https://github.com/apache/flink/pull/13509#discussion_r497297165



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/environment/StreamExecutionEnvironment.java
##########
@@ -719,12 +719,24 @@ public void registerType(Class<?> type) {
         * time, event time, or ingestion time.
         *
         * <p>If you set the characteristic to IngestionTime of EventTime this 
will set a default
-        * watermark update interval of 200 ms. If this is not applicable for 
your application
-        * you should change it using {@link 
ExecutionConfig#setAutoWatermarkInterval(long)}.
+        * watermark update interval of 200 ms. If this is not applicable for 
your application you
+        * should change it using {@link 
ExecutionConfig#setAutoWatermarkInterval(long)}.
         *
         * @param characteristic The time characteristic.
+        *
+        * @deprecated In Flink 1.12 the default stream time characteristic has 
been changed to {@link
+        *        TimeCharacteristic#EventTime}, thus you don't need to call 
this method for enabling
+        *              event-time support anymore. Explicitly using 
processing-time windows and timers works in
+        *              event-time mode. If you need to disable watermarks, 
please use {@link
+        *        ExecutionConfig#setAutoWatermarkInterval(long)}. If you are 
using {@link

Review comment:
       I think it is useful to also mention explicitly in the doc the value to 
set the watermark interval in order to use `ProcessingTime`. Sth like: "If you 
need to disable watermarks, please set the {@code watermarkInterval} to 0 using 
the {@link ExecutionConfig#setAutoWatermarkInterval(long)}."

##########
File path: 
flink-core/src/main/java/org/apache/flink/api/common/ExecutionConfig.java
##########
@@ -126,7 +126,7 @@
        private boolean autoTypeRegistrationEnabled = true;
 
        private boolean forceAvro = false;
-       private long autoWatermarkInterval = 0;
+       private long autoWatermarkInterval = 200;

Review comment:
       nit: Make the `200` a static var with a descriptive name instead of a 
"magic" number.

##########
File path: 
flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/sideoutput/SideOutputExample.java
##########
@@ -148,4 +156,29 @@ public void processElement(
 
                }
        }
+
+       /**
+        * This {@link WatermarkStrategy} assigns the current system time as 
the event-time timestamp.
+        * In a real use case you should use proper timestamps and an 
appropriate {@link
+        * WatermarkStrategy}.
+        */
+       private static class IngestionTimeWatermarkStrategy<T> implements 
WatermarkStrategy<T> {

Review comment:
       This `WatermarkStrategy` seems to be used extensively in the tests. 
Couldn't we extract it somewhere and make it reusable?




----------------------------------------------------------------
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.

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to