You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by zh...@apache.org on 2020/09/07 15:00:26 UTC

[flink] branch release-1.11 updated: [FLINK-19109][task] Ignore isLoopRunning in MailboxExecutor.isIdle

This is an automated email from the ASF dual-hosted git repository.

zhijiang pushed a commit to branch release-1.11
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/release-1.11 by this push:
     new bae7d4b  [FLINK-19109][task] Ignore isLoopRunning in MailboxExecutor.isIdle
bae7d4b is described below

commit bae7d4b7d821cc9cca3d17e1f432be22cd7dbf76
Author: Roman Khachatryan <kh...@gmail.com>
AuthorDate: Wed Sep 2 12:59:25 2020 +0200

    [FLINK-19109][task] Ignore isLoopRunning in MailboxExecutor.isIdle
    
    When closing, this flag is set, but mailbox can still contain
    (and receive new) mails (e.g. from timers) that should be processed.
    In particular, this check currently prevents periodic watermarks from
    being emitted.
---
 .../runtime/tasks/mailbox/MailboxExecutorImpl.java |   3 +-
 .../tasks/mailbox/MailboxExecutorImplTest.java     |  20 ++++
 .../streaming/api/FileReadingWatermarkITCase.java  | 118 +++++++++++++++++++++
 3 files changed, 139 insertions(+), 2 deletions(-)

diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java
index 48b57d8..697ae2c 100644
--- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java
+++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImpl.java
@@ -57,8 +57,7 @@ public final class MailboxExecutorImpl implements MailboxExecutor {
 	}
 
 	public boolean isIdle() {
-		return !mailboxProcessor.isMailboxLoopRunning() ||
-			(mailboxProcessor.isDefaultActionUnavailable() && !mailbox.hasMail() && mailbox.getState().isAcceptingMails());
+		return mailboxProcessor.isDefaultActionUnavailable() && !mailbox.hasMail() && mailbox.getState().isAcceptingMails();
 	}
 
 	@Override
diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java
index ab40979..c2a33e2 100644
--- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java
+++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/mailbox/MailboxExecutorImplTest.java
@@ -80,6 +80,26 @@ public class MailboxExecutorImplTest {
 	}
 
 	@Test
+	public void testIsIdle() throws Exception {
+		MailboxProcessor processor = new MailboxProcessor(MailboxDefaultAction.Controller::suspendDefaultAction);
+		MailboxExecutorImpl executor = (MailboxExecutorImpl) processor.getMailboxExecutor(DEFAULT_PRIORITY);
+
+		assertFalse(executor.isIdle());
+
+		processor.runMailboxStep(); // suspend default action after suspension
+		processor.mailbox.drain(); // drop any control mails
+
+		assertTrue(executor.isIdle());
+
+		executor.execute(() -> {}, "");
+		assertFalse(executor.isIdle());
+
+		processor.mailbox.drain();
+		processor.mailbox.quiesce();
+		assertFalse(executor.isIdle());
+	}
+
+	@Test
 	public void testOperations() throws Exception {
 		AtomicBoolean wasExecuted = new AtomicBoolean(false);
 		CompletableFuture.runAsync(() -> mailboxExecutor.execute(() -> wasExecuted.set(true), ""), otherThreadExecutor).get();
diff --git a/flink-tests/src/test/java/org/apache/flink/test/streaming/api/FileReadingWatermarkITCase.java b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/FileReadingWatermarkITCase.java
new file mode 100644
index 0000000..e6d3a04
--- /dev/null
+++ b/flink-tests/src/test/java/org/apache/flink/test/streaming/api/FileReadingWatermarkITCase.java
@@ -0,0 +1,118 @@
+/*
+ * 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;
+
+import org.apache.flink.api.common.JobExecutionResult;
+import org.apache.flink.api.common.accumulators.IntCounter;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.streaming.api.TimeCharacteristic;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.functions.sink.RichSinkFunction;
+import org.apache.flink.streaming.api.functions.sink.SinkFunction;
+import org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor;
+import org.apache.flink.streaming.api.windowing.time.Time;
+
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.io.PrintWriter;
+import java.util.UUID;
+
+import static org.apache.flink.util.Preconditions.checkState;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Tests that watermarks are emitted while file is being read, particularly the last split.
+ *
+ * @see <a href="https://issues.apache.org/jira/browse/FLINK-19109">FLINK-19109</a>
+ */
+public class FileReadingWatermarkITCase {
+	private static final String NUM_WATERMARKS_ACC_NAME = "numWatermarks";
+	private static final int FILE_SIZE_LINES = 100_000;
+	private static final int WATERMARK_INTERVAL_MILLIS = 10;
+	private static final int MIN_EXPECTED_WATERMARKS = 5;
+
+	@Test
+	public void testWatermarkEmissionWithChaining() throws Exception {
+		StreamExecutionEnvironment env = StreamExecutionEnvironment.createLocalEnvironment(1);
+		env.setStreamTimeCharacteristic(TimeCharacteristic.EventTime);
+		env.getConfig().setAutoWatermarkInterval(WATERMARK_INTERVAL_MILLIS);
+
+		checkState(env.isChainingEnabled());
+
+		env
+			.readTextFile(getSourceFile().getAbsolutePath())
+			.assignTimestampsAndWatermarks(getExtractorAssigner())
+			.addSink(getWatermarkCounter());
+
+		JobExecutionResult result = env.execute();
+
+		int actual = result.getAccumulatorResult(NUM_WATERMARKS_ACC_NAME);
+
+		assertTrue("too few watermarks emitted: " + actual, actual >= MIN_EXPECTED_WATERMARKS);
+	}
+
+	private File getSourceFile() throws IOException {
+		File file = File.createTempFile(UUID.randomUUID().toString(), null);
+		try (PrintWriter printWriter = new PrintWriter(file)) {
+			for (int i = 0; i < FILE_SIZE_LINES; i++) {
+				printWriter.println(i);
+			}
+		}
+		file.deleteOnExit();
+		return file;
+	}
+
+	private static BoundedOutOfOrdernessTimestampExtractor<String> getExtractorAssigner() {
+		return new BoundedOutOfOrdernessTimestampExtractor<String>(Time.hours(1)) {
+			private final long started = System.currentTimeMillis();
+
+			@Override
+			public long extractTimestamp(String line) {
+				return started + Long.parseLong(line);
+			}
+		};
+	}
+
+	private static SinkFunction<String> getWatermarkCounter() {
+		return new RichSinkFunction<String>() {
+			private final IntCounter numWatermarks = new IntCounter();
+			private long lastWatermark = -1;
+
+			@Override
+			public void open(Configuration parameters) throws Exception {
+				super.open(parameters);
+				getRuntimeContext().addAccumulator(NUM_WATERMARKS_ACC_NAME, numWatermarks);
+			}
+
+			@Override
+			public void close() throws Exception {
+				super.close();
+			}
+
+			@Override
+			public void invoke(String value, SinkFunction.Context context) {
+				if (context.currentWatermark() != lastWatermark) {
+					lastWatermark = context.currentWatermark();
+					numWatermarks.add(1);
+				}
+			}
+		};
+	}
+}