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



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SourceTransformationTranslator.java
##########
@@ -36,26 +38,29 @@
  * @param <OUT> The type of the elements that this source produces.
  */
 @Internal
-public class SourceTransformationTranslator<OUT>
-               extends SimpleTransformationTranslator<OUT, 
SourceTransformation<OUT>> {
+public class SourceTransformationTranslator<OUT, SplitT extends SourceSplit, 
EnumChkT>
+               extends SimpleTransformationTranslator<OUT, 
SourceTransformation<OUT, SplitT, EnumChkT>> {
 
        @Override
        protected Collection<Integer> translateForBatchInternal(
-                       final SourceTransformation<OUT> transformation,
+                       final SourceTransformation<OUT, SplitT, EnumChkT> 
transformation,
                        final Context context) {
-               return translateInternal(transformation, context);
+
+               return translateInternal(transformation, context, false /* emit 
progressive watermarks */);
        }
 
        @Override
        protected Collection<Integer> translateForStreamingInternal(
-                       final SourceTransformation<OUT> transformation,
+                       final SourceTransformation<OUT, SplitT, EnumChkT> 
transformation,
                        final Context context) {
-               return translateInternal(transformation, context);
+
+               return translateInternal(transformation, context, true /* emit 
progressive watermarks */);
        }
 
        private Collection<Integer> translateInternal(
-                       final SourceTransformation<OUT> transformation,
-                       final Context context) {
+                       final SourceTransformation<OUT, SplitT, EnumChkT> 
transformation,
+                       final Context context,
+                       boolean emitProgressiveWatermarks) {

Review comment:
       Why not simply `emitWatermarks`?

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SourceTransformationTranslator.java
##########
@@ -36,26 +38,29 @@
  * @param <OUT> The type of the elements that this source produces.
  */
 @Internal
-public class SourceTransformationTranslator<OUT>
-               extends SimpleTransformationTranslator<OUT, 
SourceTransformation<OUT>> {
+public class SourceTransformationTranslator<OUT, SplitT extends SourceSplit, 
EnumChkT>
+               extends SimpleTransformationTranslator<OUT, 
SourceTransformation<OUT, SplitT, EnumChkT>> {
 
        @Override
        protected Collection<Integer> translateForBatchInternal(
-                       final SourceTransformation<OUT> transformation,
+                       final SourceTransformation<OUT, SplitT, EnumChkT> 
transformation,
                        final Context context) {
-               return translateInternal(transformation, context);
+
+               return translateInternal(transformation, context, false /* emit 
progressive watermarks */);

Review comment:
       Update the comment to reflect that we do not send Watermarks in this 
case.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SourceOperatorFactory.java
##########
@@ -51,19 +51,36 @@ Licensed to the Apache Software Foundation (ASF) under one
        /** The event time setup (timestamp assigners, watermark generators, 
etc.). */
        private final WatermarkStrategy<OUT> watermarkStrategy;
 
+       /**
+        * Whether to periodically emit watermarks as we go or only one final 
watermark at the end of
+        * input.
+        */
+       private final boolean emitProgressiveWatermarks;

Review comment:
       Why not simply `emitWatermarks`?




----------------------------------------------------------------
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:
us...@infra.apache.org


Reply via email to