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



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/MultipleInputStreamTask.java
##########
@@ -18,32 +18,48 @@
 package org.apache.flink.streaming.runtime.tasks;
 
 import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointMetaData;
+import org.apache.flink.runtime.checkpoint.CheckpointMetrics;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo;
 import org.apache.flink.runtime.execution.Environment;
+import org.apache.flink.runtime.io.network.api.CheckpointBarrier;
 import org.apache.flink.runtime.io.network.partition.consumer.IndexedInputGate;
 import org.apache.flink.runtime.metrics.MetricNames;
 import org.apache.flink.streaming.api.graph.StreamConfig;
 import org.apache.flink.streaming.api.graph.StreamConfig.InputConfig;
 import org.apache.flink.streaming.api.graph.StreamEdge;
 import org.apache.flink.streaming.api.operators.InputSelectable;
 import org.apache.flink.streaming.api.operators.MultipleInputStreamOperator;
+import org.apache.flink.streaming.runtime.io.CheckpointBarrierHandler;
 import org.apache.flink.streaming.runtime.io.CheckpointedInputGate;
 import org.apache.flink.streaming.runtime.io.InputProcessorUtil;
 import org.apache.flink.streaming.runtime.io.MultipleInputSelectionHandler;
 import org.apache.flink.streaming.runtime.io.StreamMultipleInputProcessor;
+import org.apache.flink.streaming.runtime.io.StreamTaskSourceInput;
 import org.apache.flink.streaming.runtime.metrics.MinWatermarkGauge;
 import org.apache.flink.streaming.runtime.metrics.WatermarkGauge;
 
+import javax.annotation.Nullable;
+
+import java.io.IOException;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
-
-import static org.apache.flink.util.Preconditions.checkState;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Future;
 
 /**
  * A {@link StreamTask} for executing a {@link MultipleInputStreamOperator} 
and supporting
  * the {@link MultipleInputStreamOperator} to select input for reading.
  */
 @Internal
 public class MultipleInputStreamTask<OUT> extends StreamTask<OUT, 
MultipleInputStreamOperator<OUT>> {
+       private final HashMap<Long, CompletableFuture<Boolean>> 
pendingCheckpointCompletedFutures = new HashMap<>();

Review comment:
       It’s even worse. The future is currently used only for tests…
   
   I’ve tried to refactor it in someway to avoid it for the 
MultipleInputStreamTask, but it turned out more difficult then implementing 
this Map.
   
   And yes, currently `triggerOnBarrier` happens always after 
`triggerCheckpointAsync`. 
   
   I also reasoned that if we accumulated some garbage in case of some problem 
with a cancellation/checkpoint failure that I might have missed, it shouldn't 
be a big deal, as long it's rare? But maybe we can cap it's size to 100 * 
`CheckpointConfig#getMaxConcurrentCheckpoints`?
   
   However it might be better to get rid of the future somehow... 
   
   What do you think? Do you have some idea?
   
   
   
   
   
   
   




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