rkhachatryan commented on a change in pull request #13351: URL: https://github.com/apache/flink/pull/13351#discussion_r494498226
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/StreamTask.java ########## @@ -502,33 +503,24 @@ protected void beforeInvoke() throws Exception { } private void readRecoveredChannelState() throws IOException, InterruptedException { - ChannelStateReader reader = getEnvironment().getTaskStateManager().getChannelStateReader(); - if (!reader.hasChannelStates()) { - requestPartitions(); - return; - } - - ResultPartitionWriter[] writers = getEnvironment().getAllWriters(); - if (writers != null) { - for (ResultPartitionWriter writer : writers) { - if (writer instanceof CheckpointedResultPartition) { - ((CheckpointedResultPartition) writer).readRecoveredState(reader); - } else { - throw new IllegalStateException( - "Cannot restore state to a non-checkpointable partition type: " + writer); + SequentialChannelStateReader reader = getEnvironment().getTaskStateManager().getSequentialChannelStateReader(); + if (reader.hasChannelStates()) { + reader.readOutputData(getEnvironment().getAllWriters()); + channelIOExecutor.execute(() -> { + try { + reader.readInputData(getEnvironment().getAllInputGates()); + } catch (Exception e) { + asyncExceptionHandler.handleAsyncException("Unable to read channel state", e); } - } - } + }); - // It would get possible benefits to recovery input side after output side, which guarantees the - // output can request more floating buffers from global firstly. - InputGate[] inputGates = getEnvironment().getAllInputGates(); - if (inputGates != null && inputGates.length > 0) { - for (InputGate inputGate : inputGates) { + for (InputGate inputGate : getEnvironment().getAllInputGates()) { inputGate - .readRecoveredState(channelIOExecutor, reader) + .getStateConsumedFuture() .thenRun(() -> mainMailboxExecutor.execute(inputGate::requestPartitions, "Input gate request partitions")); } + } else { + requestPartitions(); Review comment: Yes, this is what I'm doing in `[FLINK-18989][task] Read channel state unconditionally` (but I'm trying to move this commit to #13467). ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { Review comment: > What is the relation between delegate and handles 1:M, that is one underlying delegate can be referenced by 1 or more handles. > we only have one state file per subtask Yes, but on downscaling we can have more. Does this answer your question? ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { + readSequentially(delegateAndHandles.getKey(), delegateAndHandles.getValue(), stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readSequentially( + StreamStateHandle streamStateHandle, + List<Handle> channelStateHandles, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + try (FSDataInputStream is = streamStateHandle.openInputStream()) { + serializer.readHeader(is); + for (Tuple2<Long, Info> offsetAndChannelInfo : extractOffsetsSorted(channelStateHandles)) { + chunkReader.readChunk(is, offsetAndChannelInfo.f0, stateHandler, offsetAndChannelInfo.f1); + } + } + } + + private Stream<OperatorSubtaskState> streamSubtaskStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().map(Map.Entry::getValue); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Map<StreamStateHandle, List<Handle>> groupByDelegate( + Stream<OperatorSubtaskState> states, + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor) { + return states + .map(stateHandleExtractor).flatMap(Collection::stream) + .peek(validate()) + .collect(groupingBy(AbstractChannelStateHandle::getDelegate)); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Consumer<Handle> validate() { + Set<Info> seen = new HashSet<>(); + // expect each channel to be described only once; otherwise, buffers in channel could be re-ordered + return handle -> Preconditions.checkState(seen.add(handle.getInfo()), "duplicate channel info: %s"); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> List<Tuple2<Long, Info>> extractOffsetsSorted(List<Handle> channelStateHandles) { + return channelStateHandles + .stream() + .flatMap(SequentialChannelStateReaderImpl::extractOffsets) + .sorted(comparingLong(offsetAndInfo -> offsetAndInfo.f0)) + .collect(toList()); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Stream<Tuple2<Long, Info>> extractOffsets(Handle handle) { + return handle.getOffsets().stream().map(offset -> Tuple2.of(offset, handle.getInfo())); + } + + @Override + public void close() throws Exception { + } + +} + +interface RecoveredChannelStateHandler<Info, Context> extends AutoCloseable { + Tuple2<ChannelStateByteBuffer, Context> getBuffer(Info info) throws IOException; + + void recover(Info info, Context context) throws IOException; +} + +class InputChannelRecoveredStateHandler implements RecoveredChannelStateHandler<InputChannelInfo, Buffer> { + private final InputGate[] inputGates; + + InputChannelRecoveredStateHandler(InputGate[] inputGates) { + this.inputGates = inputGates; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Buffer> getBuffer(InputChannelInfo channelInfo) throws IOException { + RecoveredInputChannel channel = getChannel(channelInfo); + Buffer buffer; + try { + buffer = channel.getBuffer(); + return Tuple2.of(wrap(buffer), buffer); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void recover(InputChannelInfo channelInfo, Buffer buffer) { + if (buffer.readableBytes() > 0) { + getChannel(channelInfo).onRecoveredStateBuffer(buffer); + } else { + buffer.recycleBuffer(); + } + } + + @Override + public void close() throws IOException { + // note that we need to finish all RecoveredInputChannels, not just those with state + for (final InputGate inputGate : inputGates) { + inputGate.finishReadRecoveredState(); + } + } + + private RecoveredInputChannel getChannel(InputChannelInfo info) { + return (RecoveredInputChannel) inputGates[info.getGateIdx()].getChannel(info.getInputChannelIdx()); + } +} + +class ResultSubpartitionRecoveredStateHandler implements RecoveredChannelStateHandler<ResultSubpartitionInfo, Tuple2<BufferBuilder, BufferConsumer>> { + + private final ResultPartitionWriter[] writers; + + ResultSubpartitionRecoveredStateHandler(ResultPartitionWriter[] writers) { + this.writers = writers; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Tuple2<BufferBuilder, BufferConsumer>> getBuffer(ResultSubpartitionInfo subpartitionInfo) throws IOException { + BufferBuilder bufferBuilder; + try { + bufferBuilder = getWriter(subpartitionInfo).getBufferBuilder(subpartitionInfo.getSubPartitionIdx()); + } catch (InterruptedException e) { + throw new IOException(e); + } + return Tuple2.of(wrap(bufferBuilder), Tuple2.of(bufferBuilder, bufferBuilder.createBufferConsumer())); + } + + @Override + public void recover(ResultSubpartitionInfo subpartitionInfo, Tuple2<BufferBuilder, BufferConsumer> bufferConsumer) throws IOException { + bufferConsumer.f0.finish(); + getWriter(subpartitionInfo).addBufferConsumer(bufferConsumer.f1, subpartitionInfo.getSubPartitionIdx()); + } + + private ResultPartitionWriter getWriter(ResultSubpartitionInfo info) { + return writers[info.getPartitionIdx()]; + } + + @Override + public void close() { + } +} + +class ChannelStateChunkReader { + private final ChannelStateSerializer serializer; + + ChannelStateChunkReader(ChannelStateSerializer serializer) { + this.serializer = serializer; + } + + <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readChunk( + FSDataInputStream source, + long sourceOffset, + RecoveredChannelStateHandler<Info, Context> stateHandler, + Info channelInfo) throws IOException { + if (source.getPos() != sourceOffset) { + source.seek(sourceOffset); + } + int length = serializer.readLength(source); + while (length > 0) { Review comment: length is decremented below: ```length -= serializer.readData``` ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { + readSequentially(delegateAndHandles.getKey(), delegateAndHandles.getValue(), stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readSequentially( + StreamStateHandle streamStateHandle, + List<Handle> channelStateHandles, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + try (FSDataInputStream is = streamStateHandle.openInputStream()) { + serializer.readHeader(is); + for (Tuple2<Long, Info> offsetAndChannelInfo : extractOffsetsSorted(channelStateHandles)) { + chunkReader.readChunk(is, offsetAndChannelInfo.f0, stateHandler, offsetAndChannelInfo.f1); + } + } + } + + private Stream<OperatorSubtaskState> streamSubtaskStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().map(Map.Entry::getValue); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Map<StreamStateHandle, List<Handle>> groupByDelegate( + Stream<OperatorSubtaskState> states, + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor) { + return states + .map(stateHandleExtractor).flatMap(Collection::stream) + .peek(validate()) + .collect(groupingBy(AbstractChannelStateHandle::getDelegate)); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Consumer<Handle> validate() { + Set<Info> seen = new HashSet<>(); + // expect each channel to be described only once; otherwise, buffers in channel could be re-ordered + return handle -> Preconditions.checkState(seen.add(handle.getInfo()), "duplicate channel info: %s"); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> List<Tuple2<Long, Info>> extractOffsetsSorted(List<Handle> channelStateHandles) { + return channelStateHandles + .stream() + .flatMap(SequentialChannelStateReaderImpl::extractOffsets) + .sorted(comparingLong(offsetAndInfo -> offsetAndInfo.f0)) + .collect(toList()); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Stream<Tuple2<Long, Info>> extractOffsets(Handle handle) { + return handle.getOffsets().stream().map(offset -> Tuple2.of(offset, handle.getInfo())); + } + + @Override + public void close() throws Exception { + } + +} + +interface RecoveredChannelStateHandler<Info, Context> extends AutoCloseable { + Tuple2<ChannelStateByteBuffer, Context> getBuffer(Info info) throws IOException; + + void recover(Info info, Context context) throws IOException; +} + +class InputChannelRecoveredStateHandler implements RecoveredChannelStateHandler<InputChannelInfo, Buffer> { + private final InputGate[] inputGates; + + InputChannelRecoveredStateHandler(InputGate[] inputGates) { + this.inputGates = inputGates; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Buffer> getBuffer(InputChannelInfo channelInfo) throws IOException { + RecoveredInputChannel channel = getChannel(channelInfo); + Buffer buffer; + try { + buffer = channel.getBuffer(); + return Tuple2.of(wrap(buffer), buffer); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void recover(InputChannelInfo channelInfo, Buffer buffer) { + if (buffer.readableBytes() > 0) { + getChannel(channelInfo).onRecoveredStateBuffer(buffer); + } else { + buffer.recycleBuffer(); + } + } + + @Override + public void close() throws IOException { + // note that we need to finish all RecoveredInputChannels, not just those with state + for (final InputGate inputGate : inputGates) { + inputGate.finishReadRecoveredState(); + } + } + + private RecoveredInputChannel getChannel(InputChannelInfo info) { + return (RecoveredInputChannel) inputGates[info.getGateIdx()].getChannel(info.getInputChannelIdx()); + } +} + +class ResultSubpartitionRecoveredStateHandler implements RecoveredChannelStateHandler<ResultSubpartitionInfo, Tuple2<BufferBuilder, BufferConsumer>> { + + private final ResultPartitionWriter[] writers; + + ResultSubpartitionRecoveredStateHandler(ResultPartitionWriter[] writers) { + this.writers = writers; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Tuple2<BufferBuilder, BufferConsumer>> getBuffer(ResultSubpartitionInfo subpartitionInfo) throws IOException { + BufferBuilder bufferBuilder; + try { + bufferBuilder = getWriter(subpartitionInfo).getBufferBuilder(subpartitionInfo.getSubPartitionIdx()); + } catch (InterruptedException e) { + throw new IOException(e); + } + return Tuple2.of(wrap(bufferBuilder), Tuple2.of(bufferBuilder, bufferBuilder.createBufferConsumer())); + } + + @Override + public void recover(ResultSubpartitionInfo subpartitionInfo, Tuple2<BufferBuilder, BufferConsumer> bufferConsumer) throws IOException { + bufferConsumer.f0.finish(); + getWriter(subpartitionInfo).addBufferConsumer(bufferConsumer.f1, subpartitionInfo.getSubPartitionIdx()); + } + + private ResultPartitionWriter getWriter(ResultSubpartitionInfo info) { + return writers[info.getPartitionIdx()]; + } + + @Override + public void close() { + } +} + +class ChannelStateChunkReader { Review comment: Yes, mostly for testing. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { + readSequentially(delegateAndHandles.getKey(), delegateAndHandles.getValue(), stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readSequentially( + StreamStateHandle streamStateHandle, + List<Handle> channelStateHandles, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + try (FSDataInputStream is = streamStateHandle.openInputStream()) { + serializer.readHeader(is); + for (Tuple2<Long, Info> offsetAndChannelInfo : extractOffsetsSorted(channelStateHandles)) { + chunkReader.readChunk(is, offsetAndChannelInfo.f0, stateHandler, offsetAndChannelInfo.f1); + } + } + } + + private Stream<OperatorSubtaskState> streamSubtaskStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().map(Map.Entry::getValue); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Map<StreamStateHandle, List<Handle>> groupByDelegate( + Stream<OperatorSubtaskState> states, + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor) { + return states + .map(stateHandleExtractor).flatMap(Collection::stream) + .peek(validate()) + .collect(groupingBy(AbstractChannelStateHandle::getDelegate)); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Consumer<Handle> validate() { + Set<Info> seen = new HashSet<>(); + // expect each channel to be described only once; otherwise, buffers in channel could be re-ordered + return handle -> Preconditions.checkState(seen.add(handle.getInfo()), "duplicate channel info: %s"); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> List<Tuple2<Long, Info>> extractOffsetsSorted(List<Handle> channelStateHandles) { + return channelStateHandles + .stream() + .flatMap(SequentialChannelStateReaderImpl::extractOffsets) + .sorted(comparingLong(offsetAndInfo -> offsetAndInfo.f0)) + .collect(toList()); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Stream<Tuple2<Long, Info>> extractOffsets(Handle handle) { + return handle.getOffsets().stream().map(offset -> Tuple2.of(offset, handle.getInfo())); + } + + @Override + public void close() throws Exception { + } + +} + +interface RecoveredChannelStateHandler<Info, Context> extends AutoCloseable { + Tuple2<ChannelStateByteBuffer, Context> getBuffer(Info info) throws IOException; + + void recover(Info info, Context context) throws IOException; +} + +class InputChannelRecoveredStateHandler implements RecoveredChannelStateHandler<InputChannelInfo, Buffer> { + private final InputGate[] inputGates; + + InputChannelRecoveredStateHandler(InputGate[] inputGates) { + this.inputGates = inputGates; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Buffer> getBuffer(InputChannelInfo channelInfo) throws IOException { + RecoveredInputChannel channel = getChannel(channelInfo); + Buffer buffer; + try { + buffer = channel.getBuffer(); + return Tuple2.of(wrap(buffer), buffer); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void recover(InputChannelInfo channelInfo, Buffer buffer) { + if (buffer.readableBytes() > 0) { + getChannel(channelInfo).onRecoveredStateBuffer(buffer); + } else { + buffer.recycleBuffer(); + } + } + + @Override + public void close() throws IOException { + // note that we need to finish all RecoveredInputChannels, not just those with state + for (final InputGate inputGate : inputGates) { + inputGate.finishReadRecoveredState(); + } + } + + private RecoveredInputChannel getChannel(InputChannelInfo info) { + return (RecoveredInputChannel) inputGates[info.getGateIdx()].getChannel(info.getInputChannelIdx()); + } +} + +class ResultSubpartitionRecoveredStateHandler implements RecoveredChannelStateHandler<ResultSubpartitionInfo, Tuple2<BufferBuilder, BufferConsumer>> { + + private final ResultPartitionWriter[] writers; + + ResultSubpartitionRecoveredStateHandler(ResultPartitionWriter[] writers) { + this.writers = writers; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Tuple2<BufferBuilder, BufferConsumer>> getBuffer(ResultSubpartitionInfo subpartitionInfo) throws IOException { + BufferBuilder bufferBuilder; + try { + bufferBuilder = getWriter(subpartitionInfo).getBufferBuilder(subpartitionInfo.getSubPartitionIdx()); + } catch (InterruptedException e) { + throw new IOException(e); + } + return Tuple2.of(wrap(bufferBuilder), Tuple2.of(bufferBuilder, bufferBuilder.createBufferConsumer())); Review comment: Here, it's important to create consumer before writing data. From `BufferBuilder.createBufferConsumer` javadoc: > Data written to BufferBuilder before creation of BufferConsumer won't be visible for that BufferConsumer ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { + readSequentially(delegateAndHandles.getKey(), delegateAndHandles.getValue(), stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readSequentially( + StreamStateHandle streamStateHandle, + List<Handle> channelStateHandles, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + try (FSDataInputStream is = streamStateHandle.openInputStream()) { + serializer.readHeader(is); + for (Tuple2<Long, Info> offsetAndChannelInfo : extractOffsetsSorted(channelStateHandles)) { + chunkReader.readChunk(is, offsetAndChannelInfo.f0, stateHandler, offsetAndChannelInfo.f1); + } + } + } + + private Stream<OperatorSubtaskState> streamSubtaskStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().map(Map.Entry::getValue); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Map<StreamStateHandle, List<Handle>> groupByDelegate( + Stream<OperatorSubtaskState> states, + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor) { + return states + .map(stateHandleExtractor).flatMap(Collection::stream) + .peek(validate()) + .collect(groupingBy(AbstractChannelStateHandle::getDelegate)); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Consumer<Handle> validate() { + Set<Info> seen = new HashSet<>(); + // expect each channel to be described only once; otherwise, buffers in channel could be re-ordered + return handle -> Preconditions.checkState(seen.add(handle.getInfo()), "duplicate channel info: %s"); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> List<Tuple2<Long, Info>> extractOffsetsSorted(List<Handle> channelStateHandles) { + return channelStateHandles + .stream() + .flatMap(SequentialChannelStateReaderImpl::extractOffsets) + .sorted(comparingLong(offsetAndInfo -> offsetAndInfo.f0)) + .collect(toList()); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Stream<Tuple2<Long, Info>> extractOffsets(Handle handle) { + return handle.getOffsets().stream().map(offset -> Tuple2.of(offset, handle.getInfo())); + } + + @Override + public void close() throws Exception { + } + +} + +interface RecoveredChannelStateHandler<Info, Context> extends AutoCloseable { + Tuple2<ChannelStateByteBuffer, Context> getBuffer(Info info) throws IOException; + + void recover(Info info, Context context) throws IOException; +} + +class InputChannelRecoveredStateHandler implements RecoveredChannelStateHandler<InputChannelInfo, Buffer> { + private final InputGate[] inputGates; + + InputChannelRecoveredStateHandler(InputGate[] inputGates) { + this.inputGates = inputGates; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Buffer> getBuffer(InputChannelInfo channelInfo) throws IOException { + RecoveredInputChannel channel = getChannel(channelInfo); + Buffer buffer; + try { + buffer = channel.getBuffer(); + return Tuple2.of(wrap(buffer), buffer); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void recover(InputChannelInfo channelInfo, Buffer buffer) { + if (buffer.readableBytes() > 0) { + getChannel(channelInfo).onRecoveredStateBuffer(buffer); + } else { + buffer.recycleBuffer(); + } + } + + @Override + public void close() throws IOException { + // note that we need to finish all RecoveredInputChannels, not just those with state + for (final InputGate inputGate : inputGates) { + inputGate.finishReadRecoveredState(); + } + } + + private RecoveredInputChannel getChannel(InputChannelInfo info) { + return (RecoveredInputChannel) inputGates[info.getGateIdx()].getChannel(info.getInputChannelIdx()); + } +} + +class ResultSubpartitionRecoveredStateHandler implements RecoveredChannelStateHandler<ResultSubpartitionInfo, Tuple2<BufferBuilder, BufferConsumer>> { + + private final ResultPartitionWriter[] writers; + + ResultSubpartitionRecoveredStateHandler(ResultPartitionWriter[] writers) { + this.writers = writers; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Tuple2<BufferBuilder, BufferConsumer>> getBuffer(ResultSubpartitionInfo subpartitionInfo) throws IOException { + BufferBuilder bufferBuilder; + try { + bufferBuilder = getWriter(subpartitionInfo).getBufferBuilder(subpartitionInfo.getSubPartitionIdx()); + } catch (InterruptedException e) { + throw new IOException(e); + } + return Tuple2.of(wrap(bufferBuilder), Tuple2.of(bufferBuilder, bufferBuilder.createBufferConsumer())); + } + + @Override + public void recover(ResultSubpartitionInfo subpartitionInfo, Tuple2<BufferBuilder, BufferConsumer> bufferConsumer) throws IOException { + bufferConsumer.f0.finish(); + getWriter(subpartitionInfo).addBufferConsumer(bufferConsumer.f1, subpartitionInfo.getSubPartitionIdx()); + } + + private ResultPartitionWriter getWriter(ResultSubpartitionInfo info) { + return writers[info.getPartitionIdx()]; + } + + @Override + public void close() { + } +} + +class ChannelStateChunkReader { + private final ChannelStateSerializer serializer; + + ChannelStateChunkReader(ChannelStateSerializer serializer) { + this.serializer = serializer; + } + + <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readChunk( + FSDataInputStream source, + long sourceOffset, + RecoveredChannelStateHandler<Info, Context> stateHandler, + Info channelInfo) throws IOException { + if (source.getPos() != sourceOffset) { + source.seek(sourceOffset); + } + int length = serializer.readLength(source); + while (length > 0) { + Tuple2<ChannelStateByteBuffer, Context> bufferWithContext = stateHandler.getBuffer(channelInfo); + try { + while (length > 0 && bufferWithContext.f0.isWritable()) { + length -= serializer.readData(source, bufferWithContext.f0, length); + } + } catch (Exception e) { + bufferWithContext.f0.recycle(); Review comment: It's enough to clean buffer, context is not supposed to hold any resources on its own. ########## File path: flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/SequentialChannelStateReaderImpl.java ########## @@ -0,0 +1,263 @@ +/* + * 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.api.java.tuple.Tuple2; +import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.checkpoint.StateObjectCollection; +import org.apache.flink.runtime.checkpoint.TaskStateSnapshot; +import org.apache.flink.runtime.io.network.api.writer.ResultPartitionWriter; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.buffer.BufferBuilder; +import org.apache.flink.runtime.io.network.buffer.BufferConsumer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.io.network.partition.consumer.RecoveredInputChannel; +import org.apache.flink.runtime.state.AbstractChannelStateHandle; +import org.apache.flink.runtime.state.StreamStateHandle; +import org.apache.flink.util.Preconditions; + +import java.io.IOException; +import java.util.Collection; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.function.Function; +import java.util.stream.Stream; + +import static java.util.Comparator.comparingLong; +import static java.util.stream.Collectors.groupingBy; +import static java.util.stream.Collectors.toList; +import static org.apache.flink.runtime.checkpoint.channel.ChannelStateByteBuffer.wrap; + +/** + * {@link SequentialChannelStateReader} implementation. + */ +public class SequentialChannelStateReaderImpl implements SequentialChannelStateReader { + + private final TaskStateSnapshot taskStateSnapshot; + private final ChannelStateSerializer serializer; + private final ChannelStateChunkReader chunkReader; + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot) { + this(taskStateSnapshot, new ChannelStateSerializerImpl()); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = new ChannelStateChunkReader(serializer); + } + + public SequentialChannelStateReaderImpl(TaskStateSnapshot taskStateSnapshot, ChannelStateSerializer serializer, ChannelStateChunkReader chunkReader) { + this.taskStateSnapshot = taskStateSnapshot; + this.serializer = serializer; + this.chunkReader = chunkReader; + } + + @Override + public boolean hasChannelStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().anyMatch(subtaskStateEntry -> + subtaskStateEntry.getValue().getInputChannelState().stream().anyMatch(h -> !h.getOffsets().isEmpty()) || + subtaskStateEntry.getValue().getResultSubpartitionState().stream().anyMatch(h -> !h.getOffsets().isEmpty())); + } + + @Override + public void readInputData(InputGate[] inputGates) throws IOException { + try (InputChannelRecoveredStateHandler stateHandler = new InputChannelRecoveredStateHandler(inputGates)) { + read(OperatorSubtaskState::getInputChannelState, stateHandler); + } + } + + @Override + public void readOutputData(ResultPartitionWriter[] writers) throws IOException { + try (ResultSubpartitionRecoveredStateHandler stateHandler = new ResultSubpartitionRecoveredStateHandler(writers)) { + read(OperatorSubtaskState::getResultSubpartitionState, stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void read( + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + for (Map.Entry<StreamStateHandle, List<Handle>> delegateAndHandles : groupByDelegate(streamSubtaskStates(), stateHandleExtractor).entrySet()) { + readSequentially(delegateAndHandles.getKey(), delegateAndHandles.getValue(), stateHandler); + } + } + + private <Info, Context, Handle extends AbstractChannelStateHandle<Info>> void readSequentially( + StreamStateHandle streamStateHandle, + List<Handle> channelStateHandles, + RecoveredChannelStateHandler<Info, Context> stateHandler) throws IOException { + try (FSDataInputStream is = streamStateHandle.openInputStream()) { + serializer.readHeader(is); + for (Tuple2<Long, Info> offsetAndChannelInfo : extractOffsetsSorted(channelStateHandles)) { + chunkReader.readChunk(is, offsetAndChannelInfo.f0, stateHandler, offsetAndChannelInfo.f1); + } + } + } + + private Stream<OperatorSubtaskState> streamSubtaskStates() { + return taskStateSnapshot.getSubtaskStateMappings().stream().map(Map.Entry::getValue); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Map<StreamStateHandle, List<Handle>> groupByDelegate( + Stream<OperatorSubtaskState> states, + Function<OperatorSubtaskState, StateObjectCollection<Handle>> stateHandleExtractor) { + return states + .map(stateHandleExtractor).flatMap(Collection::stream) + .peek(validate()) + .collect(groupingBy(AbstractChannelStateHandle::getDelegate)); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Consumer<Handle> validate() { + Set<Info> seen = new HashSet<>(); + // expect each channel to be described only once; otherwise, buffers in channel could be re-ordered + return handle -> Preconditions.checkState(seen.add(handle.getInfo()), "duplicate channel info: %s"); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> List<Tuple2<Long, Info>> extractOffsetsSorted(List<Handle> channelStateHandles) { + return channelStateHandles + .stream() + .flatMap(SequentialChannelStateReaderImpl::extractOffsets) + .sorted(comparingLong(offsetAndInfo -> offsetAndInfo.f0)) + .collect(toList()); + } + + private static <Info, Handle extends AbstractChannelStateHandle<Info>> Stream<Tuple2<Long, Info>> extractOffsets(Handle handle) { + return handle.getOffsets().stream().map(offset -> Tuple2.of(offset, handle.getInfo())); + } + + @Override + public void close() throws Exception { + } + +} + +interface RecoveredChannelStateHandler<Info, Context> extends AutoCloseable { + Tuple2<ChannelStateByteBuffer, Context> getBuffer(Info info) throws IOException; + + void recover(Info info, Context context) throws IOException; +} + +class InputChannelRecoveredStateHandler implements RecoveredChannelStateHandler<InputChannelInfo, Buffer> { + private final InputGate[] inputGates; + + InputChannelRecoveredStateHandler(InputGate[] inputGates) { + this.inputGates = inputGates; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Buffer> getBuffer(InputChannelInfo channelInfo) throws IOException { + RecoveredInputChannel channel = getChannel(channelInfo); + Buffer buffer; + try { + buffer = channel.getBuffer(); + return Tuple2.of(wrap(buffer), buffer); + } catch (InterruptedException e) { + throw new IOException(e); + } + } + + @Override + public void recover(InputChannelInfo channelInfo, Buffer buffer) { + if (buffer.readableBytes() > 0) { + getChannel(channelInfo).onRecoveredStateBuffer(buffer); + } else { + buffer.recycleBuffer(); + } + } + + @Override + public void close() throws IOException { + // note that we need to finish all RecoveredInputChannels, not just those with state + for (final InputGate inputGate : inputGates) { + inputGate.finishReadRecoveredState(); + } + } + + private RecoveredInputChannel getChannel(InputChannelInfo info) { + return (RecoveredInputChannel) inputGates[info.getGateIdx()].getChannel(info.getInputChannelIdx()); + } +} + +class ResultSubpartitionRecoveredStateHandler implements RecoveredChannelStateHandler<ResultSubpartitionInfo, Tuple2<BufferBuilder, BufferConsumer>> { + + private final ResultPartitionWriter[] writers; + + ResultSubpartitionRecoveredStateHandler(ResultPartitionWriter[] writers) { + this.writers = writers; + } + + @Override + public Tuple2<ChannelStateByteBuffer, Tuple2<BufferBuilder, BufferConsumer>> getBuffer(ResultSubpartitionInfo subpartitionInfo) throws IOException { + BufferBuilder bufferBuilder; + try { + bufferBuilder = getWriter(subpartitionInfo).getBufferBuilder(subpartitionInfo.getSubPartitionIdx()); + } catch (InterruptedException e) { + throw new IOException(e); Review comment: `InterruptedException` is a checked exception so I'd had to declare it if not wrapping. But it's not appropriate for this layer (IO). `Thread.currentThread().interrupt();` only sets the status, so the caller can skip interrupt status check. ########## File path: flink-runtime/src/test/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateSerializerImplTest.java ########## @@ -146,4 +145,10 @@ private void readAndCheck(byte[] data, ChannelStateSerializerImpl serializer, By } } + static byte[] generateData(int len) { Review comment: Previously, this method was imported from `ChannelStateReaderImplTest`. In this commit `ChannelStateReaderImplTest` was removed and `generateData` moved to this class. ---------------------------------------------------------------- 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