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 2020/08/10 14:45:52 UTC

[GitHub] [flink] pnowojski commented on a change in pull request #10939: [FLINK-15750][network] Moving output flushing to the mailbox

pnowojski commented on a change in pull request #10939:
URL: https://github.com/apache/flink/pull/10939#discussion_r467935009



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -1451,7 +1485,9 @@ private void checkpointStreamOperator(StreamOperator<?> op) throws Exception {
 
 	private static <OUT> List<RecordWriter<SerializationDelegate<StreamRecord<OUT>>>> createRecordWriters(
 			StreamConfig configuration,
-			Environment environment) {
+			Environment environment,
+			OutputFlusher.OutputFlushers outputFlushers,

Review comment:
       I've cleaned this up a bit and moved `OutputFlushers` to `RecordWriterDelegate` class.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java
##########
@@ -1492,13 +1532,27 @@ private void checkpointStreamOperator(StreamOperator<?> op) throws Exception {
 
 		RecordWriter<SerializationDelegate<StreamRecord<OUT>>> output = new RecordWriterBuilder<SerializationDelegate<StreamRecord<OUT>>>()
 			.setChannelSelector(outputPartitioner)
-			.setTimeout(bufferTimeout)
-			.setTaskName(taskName)
+			.setFlushAlways(bufferTimeout == 0)
 			.build(bufferWriter);
 		output.setMetricGroup(environment.getMetricGroup().getIOMetricGroup());
+
+		if (bufferTimeout > 0) {
+			outputFlushers.addOutputFlusher(startOutputFlusher(output, taskName, bufferTimeout, mailboxExecutor));
+		}
 		return output;
 	}
 
+	private static OutputFlusher startOutputFlusher(
+			RecordWriter<?> output,
+			String taskName,
+			long bufferTimeout,
+			MailboxExecutor mailboxExecutor) {
+		Preconditions.checkArgument(bufferTimeout >= -1);

Review comment:
       True, here values -1 and 0 do not make sense, as now they are handled in a bit different way.
   `-1` means output flushing disabled
   `0` means flush always
   in both cases, there is no `OutputFlusher`, so I've adjusted the `checkArgument` and moved it to the `OutputFlusher` constructor.
   
   Values `(0, 10]` do make sense, and there are users requiring flushes that often. Actually a lot of effort was put into the code https://issues.apache.org/jira/browse/FLINK-8581 to make it work reasonably well with ~1ms `bufferTimeout` (and we are benchmarking those cases as well :) )

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/OutputFlusher.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.streaming.runtime.io;
+
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.flink.runtime.io.network.api.writer.RecordWriter;
+import org.apache.flink.streaming.api.operators.MailboxExecutor;
+import org.apache.flink.util.FlinkException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.concurrent.Future;
+
+/**
+ * A dedicated thread that periodically flushes the output buffers, to set upper latency bounds.
+ *
+ * <p>The thread is daemonic, because it is only a utility thread.
+ */
+public class OutputFlusher extends Thread {
+	/** Default name for the output flush thread, if no name with a task reference is given. */
+	@VisibleForTesting
+	public static final String OUTPUT_FLUSH_THREAD_NAME = "OutputFlusher";
+
+	private static final Logger LOG = LoggerFactory.getLogger(OutputFlusher.class);
+
+	private final RecordWriter<?> recordWriter;
+	private final long timeout;
+	private final MailboxExecutor mailboxExecutor;
+
+	private volatile boolean running = true;
+
+	public OutputFlusher(
+			RecordWriter<?> recordWriter,
+			String taskName,
+			long timeout,
+			MailboxExecutor mailboxExecutor) {
+		super(OUTPUT_FLUSH_THREAD_NAME + " for " + taskName);
+		this.recordWriter = recordWriter;
+		this.timeout = timeout;
+		this.mailboxExecutor = mailboxExecutor;
+		setDaemon(true);
+	}
+
+	public void terminate() {
+		running = false;
+		interrupt();
+	}
+
+	@Override
+	public void run() {
+		try {
+			Future<?> future = null;
+
+			while (running) {
+				try {
+					if (future != null) {

Review comment:
       I would love to remember why have I left it like that 😄 I hope this was just a mid work/leftover code and not some important todo.

##########
File path: flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/io/benchmark/LongRecordWriterThread.java
##########
@@ -63,46 +73,65 @@ public synchronized void shutdown() {
 	 * @param records
 	 * 		number of records to send
 	 */
-	public synchronized void setRecordsToSend(long records) {
-		checkState(!recordsToSend.isDone());
-		recordsToSend.complete(records);
+	public void setRecordsToSend(long records) {
+		mailboxExecutor.execute(() -> resumeAndSetRecordsToSend(records), "resumeAndSetRecordsToSend");

Review comment:
       I've reimplemented similar checks. Please check the fixup commit.
   
   Actually there was a small bug that I've discovered thanks to that. `defaultAction()` method when going to sleep, it was still attempting to send a record, and to compensate for that, the `currentRecordIteration++` was a bit wrong as well.




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