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_r402450729
########## 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) { Review comment: Yes, I can guess your previous motivation to name this. But I guess it might easy bring confusing for others at first glance. In general the variable naming should be consistent with class naming. But I also do not like the naming of `CheckpointStorageWorkerView` because it is hard to get the real semantic. So the root cause might be refactoring the class name in a hotfix commit, but i am not forcing it. Actually `CheckpointStorageWorkerView` has two purposes, if only considering the builder for factory, might be `streamFactoryBuilder/Resolver`. ---------------------------------------------------------------- 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