zhijiangW commented on a change in pull request #11515: [FLINK-16744][task] 
implement channel state persistence for unaligned checkpoints
URL: https://github.com/apache/flink/pull/11515#discussion_r402045285
 
 

 ##########
 File path: 
flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateWriterImpl.java
 ##########
 @@ -0,0 +1,225 @@
+/*
+ * 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.runtime.checkpoint.channel;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.runtime.checkpoint.CheckpointOptions;
+import org.apache.flink.runtime.io.network.buffer.Buffer;
+import org.apache.flink.runtime.state.CheckpointStorageWorkerView;
+import org.apache.flink.runtime.state.CheckpointStreamFactory;
+import org.apache.flink.util.Preconditions;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.concurrent.ThreadSafe;
+
+import java.util.Map;
+import java.util.concurrent.ArrayBlockingQueue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeInput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.completeOutput;
+import static 
org.apache.flink.runtime.checkpoint.channel.ChannelStateWriteRequest.write;
+
+/**
+ * {@link ChannelStateWriter} implemented using
+ * {@link CheckpointStreamFactory.CheckpointStateOutputStream 
CheckpointStateOutputStreams}. Internally, it has
+ * <ul>
+ * <li>one stream per checkpoint; having multiple streams would mean more 
files written and more connections opened
+ * (and more latency on restore)</li>
+ * <li>one thread; having multiple threads means more connections, couples 
with the implementation and increases complexity</li>
+ * </ul>
+ */
+@Internal
+@ThreadSafe
+public class ChannelStateWriterImpl implements ChannelStateWriter {
+
+       private static final Logger LOG = 
LoggerFactory.getLogger(ChannelStateWriterImpl.class);
+       private static final int DEFAULT_HANDOVER_CAPACITY = 10;
+       private static final int DEFAULT_MAX_CHECKPOINTS = 5; // currently, 
only single in-flight checkpoint is supported
+
+       private class ProcessRequestsLoop implements Runnable {
+               private final ChannelStateWriteRequestProcessor 
requestProcessor;
+
+               private ProcessRequestsLoop(int maxCheckpoints, 
CheckpointStorageWorkerView streamFactoryFactory, ChannelStateSerializer 
serializer) {
+                       this.requestProcessor = new 
ChannelStateWriteRequestProcessor(maxCheckpoints, streamFactoryFactory, 
serializer);
+               }
+
+               @Override
+               public void run() {
+                       try {
+                               loop();
+                       } catch (Exception ex) {
+                               thrown = ex;
+                       } finally {
+                               handover.clear();
+                               requestProcessor.cleanup(thrown != null ? 
thrown : new RuntimeException("loop terminated, isRunning: " + isRunning));
+                       }
+                       LOG.debug("loop terminated");
+               }
+
+               private void loop() throws Exception {
+                       while (isRunning || !handover.isEmpty()) {
+                               try {
+                                       
requestProcessor.processRequest(handover.take());
+                               } catch (InterruptedException e) {
+                                       if (isRunning || !handover.isEmpty()) {
+                                               LOG.info("interrupted while 
waiting for an item (continue waiting)", e);
+                                       } else {
+                                               
Thread.currentThread().interrupt();
+                                               return;
+                                       }
+                               }
+                       }
+               }
+       }
+
+       private final Thread asyncWriter;
+       private final BlockingQueue<ChannelStateWriteRequest> handover;
+       private final Map<Long, ChannelStateWriteResult> results;
+       private final int maxCheckpoints;
+       private volatile boolean isRunning;
+       private volatile Exception thrown;
+
+       public ChannelStateWriterImpl(CheckpointStorageWorkerView 
streamFactoryFactory) {
+               this(streamFactoryFactory, DEFAULT_MAX_CHECKPOINTS);
+       }
+
+       /**
+        * Creates a {@link ChannelStateWriterImpl} with {@link 
#DEFAULT_MAX_CHECKPOINTS} as {@link #maxCheckpoints}.
+        */
+       ChannelStateWriterImpl(CheckpointStorageWorkerView 
streamFactoryFactory, int maxCheckpoints) {
+               this(streamFactoryFactory, DEFAULT_HANDOVER_CAPACITY, new 
ChannelStateSerializerImpl(), maxCheckpoints);
+       }
+
+       /**
+        * Creates a {@link ChannelStateWriterImpl}.
+        *
+        * @param maxCheckpoints maximum number of checkpoints to be written 
currently or finished but not taken yet.
+        */
+       ChannelStateWriterImpl(
+                       CheckpointStorageWorkerView streamFactory,
+                       int handoverCapacity,
+                       ChannelStateSerializer serializer,
+                       int maxCheckpoints) {
+               this.handover = new ArrayBlockingQueue<>(handoverCapacity);
+               this.results = new ConcurrentHashMap<>(maxCheckpoints);
+               this.maxCheckpoints = maxCheckpoints;
+               this.asyncWriter = new Thread(new 
ProcessRequestsLoop(maxCheckpoints, streamFactory, serializer));
+               this.isRunning = true;
+       }
+
+       @Override
+       public void start(long checkpointId, CheckpointOptions 
checkpointOptions) {
+               LOG.debug("start checkpoint {} ({})", checkpointId, 
checkpointOptions);
+               rethrow();
+               Preconditions.checkState(isRunning && asyncWriter.isAlive(), 
"not running");
+               ChannelStateWriteResult result = new ChannelStateWriteResult();
+               ChannelStateWriteResult put = 
results.computeIfAbsent(checkpointId, id -> {
+                       Preconditions.checkArgument(results.size() < 
maxCheckpoints, "results.size() > maxCheckpoints", results.size(), 
maxCheckpoints);
+                       enqueue(new CheckpointStartRequest(checkpointId, 
result, checkpointOptions.getTargetLocation()));
+                       return result;
+               });
+               Preconditions.checkArgument(put == result, "result future 
already present for checkpoint id: " + checkpointId);
+       }
+
+       @Override
+       public void addInputData(long checkpointId, InputChannelInfo info, int 
startSeqNum, Buffer... data) {
+               LOG.debug("add input data, checkpoint id: {}, channel: {}, 
startSeqNum: {}, num buffers: {}",
+                       checkpointId, info, startSeqNum, data == null ? 0 : 
data.length);
+               enqueue(write(checkpointId, info, checkBufferType(data)));
+       }
+
+       @Override
+       public void addOutputData(long checkpointId, ResultSubpartitionInfo 
info, int startSeqNum, Buffer... data) {
+               LOG.debug("add output data, checkpoint id: {}, channel: {}, 
startSeqNum: {}, num buffers: {}",
+                       checkpointId, info, startSeqNum, data == null ? 0 : 
data.length);
+               enqueue(write(checkpointId, info, checkBufferType(data)));
+       }
+
+       @Override
+       public void finishInput(long checkpointId) {
+               LOG.debug("finish input data, checkpoint id: {}", checkpointId);
+               enqueue(completeInput(checkpointId));
+       }
+
+       @Override
+       public void finishOutput(long checkpointId) {
+               LOG.debug("finish output data, checkpoint id: {}", 
checkpointId);
+               enqueue(completeOutput(checkpointId));
+       }
+
+       @Override
+       public ChannelStateWriteResult getWriteResult(long checkpointId) {
+               LOG.debug("requested write result, checkpoint id: {}", 
checkpointId);
+               ChannelStateWriteResult result = results.remove(checkpointId);
+               Preconditions.checkArgument(result != null, "channel state 
write result not found for checkpoint id " + checkpointId);
+               return result;
+       }
+
+       public void open() {
+               Preconditions.checkState(isRunning);
+               asyncWriter.start();
+       }
+
+       @Override
+       public void close() {
+               isRunning = false;
+               results.clear();
+               asyncWriter.interrupt();
+               while (asyncWriter.isAlive()) {
+                       try {
+                               asyncWriter.join();
+                       } catch (InterruptedException e) {
+                               if (!asyncWriter.isAlive()) {
+                                       Thread.currentThread().interrupt();
+                               }
+                               LOG.debug("interrupted while waiting for the 
writer thread to die", e);
+                       }
+               }
+               rethrow();
+       }
+
+       private void enqueue(ChannelStateWriteRequest request) {
+               rethrow();
+               Preconditions.checkState(isRunning);
+               try {
+                       handover.put(request);
+               } catch (InterruptedException e) {
+                       throw new RuntimeException("Interrupted while trying to 
add new handover request", e);
+               }
+       }
+
+       private void rethrow() {
+               if (thrown != null) {
+                       throw new RuntimeException(thrown); // wrap to record 
current stack-trace
+               }
+       }
+
+       private static Buffer[] checkBufferType(Buffer... data) {
 
 Review comment:
   Give some javadoc to explain why we do not support event data?

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


With regards,
Apache Git Services

Reply via email to