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 2022/07/04 16:35:00 UTC

[GitHub] [flink] pnowojski opened a new pull request, #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

pnowojski opened a new pull request, #20158:
URL: https://github.com/apache/flink/pull/20158

   Otherwise tasks can not correctly determine number of inputs. This was causing an
   issue where restored as finished OneInputStreamTask was waiting for MAX_WATERMARK
   from two inputs, where the second one was null.
   
   ## Does this pull request potentially affect one of the following parts:
   
     - Dependencies (does it add or upgrade a dependency): (yes / **no**)
     - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
     - The serializers: (yes / **no** / don't know)
     - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
     - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Kubernetes/Yarn, ZooKeeper: (yes / **no** / don't know)
     - The S3 file system connector: (yes / no ****/ don't know)
   
   ## Documentation
   
     - Does this pull request introduce a new feature? (yes / **no**)
     - If yes, how is the feature documented? (**not applicable** / docs / JavaDocs / not documented)
   


-- 
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 diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914661704


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -265,17 +266,19 @@ public void setOperatorDescription(String operatorDescription) {
 
     public void setSerializersIn(TypeSerializer<?>... typeSerializersIn) {
         checkArgument(typeSerializersIn.length > 0);
-        this.typeSerializersIn = typeSerializersIn;
+        // Unfortunately code above assumes type serializer can be null, while users of for example
+        // getTypeSerializersIn would be confused by returning an array size of two with all
+        // elements set to null...
+        this.typeSerializersIn =
+                Arrays.stream(typeSerializersIn)
+                        .filter(typeSerializer -> typeSerializer != null)
+                        .toArray(TypeSerializer<?>[]::new);
     }
 
     public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   Does the code calling this method make no assumptions about the indices?
   Let's say the original list of serializers contained 2 elements, with the first one being nul
   l.
   How does the code accessing this array figure out that what is now at index 0 is what it requires?



##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -265,17 +266,19 @@ public void setOperatorDescription(String operatorDescription) {
 
     public void setSerializersIn(TypeSerializer<?>... typeSerializersIn) {
         checkArgument(typeSerializersIn.length > 0);
-        this.typeSerializersIn = typeSerializersIn;
+        // Unfortunately code above assumes type serializer can be null, while users of for example
+        // getTypeSerializersIn would be confused by returning an array size of two with all
+        // elements set to null...
+        this.typeSerializersIn =
+                Arrays.stream(typeSerializersIn)
+                        .filter(typeSerializer -> typeSerializer != null)
+                        .toArray(TypeSerializer<?>[]::new);
     }
 
     public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   Does the code calling this method make no assumptions about the indices?
   Let's say the original list of serializers contained 2 elements, with the first one being null.
   How does the code accessing this array figure out that what is now at index 0 is what it requires?



-- 
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] pnowojski commented on a diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
pnowojski commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914667914


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -265,17 +266,19 @@ public void setOperatorDescription(String operatorDescription) {
 
     public void setSerializersIn(TypeSerializer<?>... typeSerializersIn) {
         checkArgument(typeSerializersIn.length > 0);
-        this.typeSerializersIn = typeSerializersIn;
+        // Unfortunately code above assumes type serializer can be null, while users of for example
+        // getTypeSerializersIn would be confused by returning an array size of two with all
+        // elements set to null...
+        this.typeSerializersIn =
+                Arrays.stream(typeSerializersIn)
+                        .filter(typeSerializer -> typeSerializer != null)
+                        .toArray(TypeSerializer<?>[]::new);
     }
 
     public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   `StreamGraph#setSerializersFrom` was not used so I removed it in this PR :)
   
   Yes the result can be empty for source tasks.



-- 
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 diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914666576


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -265,17 +266,19 @@ public void setOperatorDescription(String operatorDescription) {
 
     public void setSerializersIn(TypeSerializer<?>... typeSerializersIn) {
         checkArgument(typeSerializersIn.length > 0);
-        this.typeSerializersIn = typeSerializersIn;
+        // Unfortunately code above assumes type serializer can be null, while users of for example
+        // getTypeSerializersIn would be confused by returning an array size of two with all
+        // elements set to null...
+        this.typeSerializersIn =
+                Arrays.stream(typeSerializersIn)
+                        .filter(typeSerializer -> typeSerializer != null)
+                        .toArray(TypeSerializer<?>[]::new);
     }
 
     public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   Can this change result in the array being empty? If so, can this conflict with `StreamGraph#setSerializersFrom`?



-- 
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] MartijnVisser commented on pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
MartijnVisser commented on PR #20158:
URL: https://github.com/apache/flink/pull/20158#issuecomment-1174738789

   @flinkbot run azure


-- 
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] pnowojski commented on a diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
pnowojski commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914665324


##########
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamNode.java:
##########
@@ -265,17 +266,19 @@ public void setOperatorDescription(String operatorDescription) {
 
     public void setSerializersIn(TypeSerializer<?>... typeSerializersIn) {
         checkArgument(typeSerializersIn.length > 0);
-        this.typeSerializersIn = typeSerializersIn;
+        // Unfortunately code above assumes type serializer can be null, while users of for example
+        // getTypeSerializersIn would be confused by returning an array size of two with all
+        // elements set to null...
+        this.typeSerializersIn =
+                Arrays.stream(typeSerializersIn)
+                        .filter(typeSerializer -> typeSerializer != null)
+                        .toArray(TypeSerializer<?>[]::new);
     }
 
     public TypeSerializer<?>[] getTypeSerializersIn() {

Review Comment:
   It must be kept in sync with `StreamNode#getInEdges`. Each edge has it's own input serializer. It can never happen that first element is null, second is not null. Sources have zero inputs, one input task has single input, two input tasks have two inputs, multi input tasks have N inputs. 



-- 
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 #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

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

   What is the root cause of a null serializer being passed in? Why can the serializer be null in the first place; does that happen for chained operators or...?


-- 
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 diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
zentol commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914722759


##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();

Review Comment:
   Do we have a requirement that emitted watermarks are monotonous?
   
   Doesn't `DateTime.now().getMillis()` by default just call `System.currentTimeMillis` anyway (so couldn't we use that directly)?



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));

Review Comment:
   ```suggestion
   ```
   These 2 are immediately run in the loop; can we remove these for deduplication purposes?



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));
+
+            while (isRunning && expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()) {
+                synchronized (sourceContext.getCheckpointLock()) {
+                    watermark = DateTime.now().getMillis();
+                    sourceContext.emitWatermark(new Watermark(watermark));
+                    if (lastSuccessfulCheckpointId == 5) {
+                        throw new RuntimeException("Force recovery");
+                    }
+                    if (lastSuccessfulCheckpointId > 10 && expectedWatermark > watermark) {
+                        expectedWatermark = watermark;

Review Comment:
   may be a matter of taste, but I'd consider a `break` here to be simpler (+ reducing the while condition to `isRunning()`).



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));
+
+            while (isRunning && expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()) {
+                synchronized (sourceContext.getCheckpointLock()) {
+                    watermark = DateTime.now().getMillis();
+                    sourceContext.emitWatermark(new Watermark(watermark));
+                    if (lastSuccessfulCheckpointId == 5) {
+                        throw new RuntimeException("Force recovery");
+                    }
+                    if (lastSuccessfulCheckpointId > 10 && expectedWatermark > watermark) {

Review Comment:
   ```suggestion
                       if (lastSuccessfulCheckpointId == 10) {
   ```
   The latter condition will usually only evaluate to true the first time it is called (excluding cases where `DateTime.now()` goes back in time).



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));
+
+            while (isRunning && expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()) {

Review Comment:
   ```suggestion
               while (isRunning) {
   ```
   I'm wondering whether this conditions does more harm than good. AFAICT it is always false after CP 10, except if a max watermark hits the sink (but if we are concerned about that case then maybe we should check that explicitly).



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */

Review Comment:
   It would be good to clarify that this test targets an issue where the job would get stuck otherwise.



##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));
+
+            while (isRunning && expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()) {
+                synchronized (sourceContext.getCheckpointLock()) {
+                    watermark = DateTime.now().getMillis();

Review Comment:
   ```suggestion
                       long watermark = DateTime.now().getMillis();
   ```



-- 
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 #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

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

   <!--
   Meta data
   {
     "version" : 1,
     "metaDataEntries" : [ {
       "hash" : "0a2c1e75426bd88efab0bbe8662005b86f9eb22d",
       "status" : "UNKNOWN",
       "url" : "TBD",
       "triggerID" : "0a2c1e75426bd88efab0bbe8662005b86f9eb22d",
       "triggerType" : "PUSH"
     } ]
   }-->
   ## CI report:
   
   * 0a2c1e75426bd88efab0bbe8662005b86f9eb22d 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] pnowojski merged pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
pnowojski merged PR #20158:
URL: https://github.com/apache/flink/pull/20158


-- 
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] pnowojski commented on pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
pnowojski commented on PR #20158:
URL: https://github.com/apache/flink/pull/20158#issuecomment-1176038063

   This code kind of always assumed serializers can be null. Both input and output. Originally, before we introduced multiple input operators, it kind of held together, as the `StreamNode` had three fields: `typeSerializerIn1`, `typeSerializerIn2` and `typeSerializerOut`. Either of those could be null. When introducing multiple inputs, I squashed the `typeSerializerInX` fields into an array, unfortunately also preserving nulls. 
   
   I've looked into disallowing nulls into those setters, but that made the caller code quite ugly, with lots of ifs. And those nulls are used in a quite a lot of places (check `StreamGraph#setSerializers(...)` calls)


-- 
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] pnowojski commented on a diff in pull request #20158: [FLINK-28357][datastream] Disallow null elements in StreamNode#typeSerializersIn

Posted by GitBox <gi...@apache.org>.
pnowojski commented on code in PR #20158:
URL: https://github.com/apache/flink/pull/20158#discussion_r914778495


##########
flink-tests/src/test/java/org/apache/flink/test/streaming/api/datastream/FinishedSourcesWatermarkITCase.java:
##########
@@ -0,0 +1,131 @@
+/*
+ * 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.test.streaming.api.datastream;
+
+import org.apache.flink.api.common.state.CheckpointListener;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.datastream.DataStream;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.co.CoProcessFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.source.RichSourceFunction;
+import org.apache.flink.streaming.api.watermark.Watermark;
+import org.apache.flink.util.Collector;
+import org.apache.flink.util.TestLogger;
+
+import org.joda.time.DateTime;
+import org.junit.Test;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/** Test that ensures watermarks are correctly propagating with finished sources. */
+public class FinishedSourcesWatermarkITCase extends TestLogger {
+
+    private static final AtomicLong ACTUAL_DOWNSTREAM_WATERMARK = new AtomicLong(0);
+
+    @Test
+    public void testTwoConsecutiveFinishedTasksShouldPropagateMaxWatermark() throws Exception {
+        Configuration conf = new Configuration();
+        StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+
+        // disable chaining to make sure we will have two consecutive checkpoints with Task ==
+        // FINISHED
+        env.disableOperatorChaining();
+        env.enableCheckpointing(100);
+
+        // create our sources - one that will want to run forever, and another that finishes
+        // immediately
+        DataStream<String> runningStreamIn =
+                env.addSource(new LongRunningSource(), "Long Running Source");
+        DataStream<String> emptyStream =
+                env.addSource(new ShortLivedEmptySource(), "Short Lived Source");
+
+        // pass the empty stream through a simple map() function
+        DataStream<String> mappedEmptyStream = emptyStream.map(v -> v).name("Empty Stream Map");
+
+        // join the two streams together to see what watermark is reached during startup and after a
+        // recovery
+        runningStreamIn
+                .connect(mappedEmptyStream)
+                .process(new MyCoProcessFunction())
+                .name("Join")
+                .addSink(
+                        new SinkFunction<String>() {
+                            @Override
+                            public void writeWatermark(
+                                    org.apache.flink.api.common.eventtime.Watermark watermark) {
+                                ACTUAL_DOWNSTREAM_WATERMARK.set(watermark.getTimestamp());
+                            }
+                        });
+
+        env.execute();
+    }
+
+    private static class LongRunningSource extends RichSourceFunction<String>
+            implements CheckpointListener {
+        private volatile boolean isRunning = true;
+        private transient long lastSuccessfulCheckpointId;
+
+        @Override
+        public void run(SourceContext<String> sourceContext) throws Exception {
+            long expectedWatermark = Watermark.MAX_WATERMARK.getTimestamp();
+            long watermark = DateTime.now().getMillis();
+            sourceContext.emitWatermark(new Watermark(watermark));
+
+            while (isRunning && expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()) {
+                synchronized (sourceContext.getCheckpointLock()) {
+                    watermark = DateTime.now().getMillis();
+                    sourceContext.emitWatermark(new Watermark(watermark));
+                    if (lastSuccessfulCheckpointId == 5) {
+                        throw new RuntimeException("Force recovery");
+                    }
+                    if (lastSuccessfulCheckpointId > 10 && expectedWatermark > watermark) {

Review Comment:
   Re this and above. This is kind of the point here. We are waiting until 10th checkpoint. On the 10th checkpoint, we are recoding what was the last emitted watermark, and remembering it to be the `expectedWatermark`. Then we are still running unknown number of iterations, until the downstream sink finally receives this `expectedWatermark`. Hence we can not get rid of `expectedWatermark > ACTUAL_DOWNSTREAM_WATERMARK.get()`.
   
   I will try to rephrase the code a bit, to make it more explicit/clear. 



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