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



##########
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` quite often should be handled differently and 
it's good to know that the `InterruptedException` has happened.
   
   For example, when caller received `IOException`, it might decide to retry or 
just keep going if that was not a critical part. `InterruptedException` should 
on the other hand shut down the code semi gracefully and as quickly as possible.
   
   Here I think adding `throws InterruptedException` would be much better. But 
I didn't look how deep in a call stack are we and how difficult would it be to 
propagate it.
   
   re `Thread.currentThread().interrupt();`, the crowd wisdom is to always set 
it when wrapping/hiding/rethrowing `InterruptedException`. I think we are using 
interrupted flag in only a couple of places, but it's better to keep it clean, 
just in case we would had to use it some important place and then we would 
discover we need to fix `interrupted` flag everywhere to make it work.




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


Reply via email to