AHeise 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_r404984628
########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateReaderImpl.java ########## @@ -0,0 +1,116 @@ +/* + * 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.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.checkpoint.channel.RefCountingFSDataInputStream.RefCountingFSDataInputStreamFactory; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.util.Preconditions; + +import org.apache.flink.shaded.guava18.com.google.common.io.Closer; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.concurrent.NotThreadSafe; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static java.util.Arrays.asList; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * {@link ChannelStateReader} implementation. Usage considerations: + * <ol> + * <li>state of a channel can be read once per instance of this class; once done it returns + * {@link org.apache.flink.runtime.checkpoint.channel.ChannelStateReader.ReadResult#NO_MORE_DATA NO_MORE_DATA}</li> + * <li>reader/writer indices of the passed buffer are respected and updated</li> + * <li>buffers must be prepared (cleared) before passing to reader</li> + * <li>buffers must be released after use</li> + * </ol> + */ +@NotThreadSafe +@Internal +public class ChannelStateReaderImpl implements ChannelStateReader { + private static final Logger log = LoggerFactory.getLogger(ChannelStateReaderImpl.class); + + private final Map<InputChannelInfo, ChannelStateStreamReader> inputChannelHandleReaders; + private final Map<ResultSubpartitionInfo, ChannelStateStreamReader> resultSubpartitionHandleReaders; + + public ChannelStateReaderImpl(TaskStateSnapshot snapshot) { + this(snapshot, new ChannelStateSerializerImpl()); + } + + @SuppressWarnings("WeakerAccess") + ChannelStateReaderImpl(TaskStateSnapshot snapshot, ChannelStateDeserializer serializer) { + RefCountingFSDataInputStreamFactory streamFactory = new RefCountingFSDataInputStreamFactory(serializer); + final HashMap<InputChannelInfo, ChannelStateStreamReader> inputChannelHandleReadersTmp = new HashMap<>(); Review comment: As long as you are not handing out `this` directly or indirectly, constructor should be atomic in terms of memory synchronization. No other thread can see this instance of `ChannelStateReaderImpl` and find any inconsistent state. ``` ChannelStateReaderImpl reader = new ChannelStateReaderImpl(...); reader.... // <- at this point all maps are good ``` ---------------------------------------------------------------- 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