AHeise commented on a change in pull request #13845: URL: https://github.com/apache/flink/pull/13845#discussion_r519228359
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/Demultiplexer.java ########## @@ -0,0 +1,402 @@ +/* + * 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.streaming.runtime.io; + +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptor; +import org.apache.flink.runtime.checkpoint.RescaledChannelsMapping; +import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo; +import org.apache.flink.runtime.io.disk.iomanager.IOManager; +import org.apache.flink.runtime.io.network.api.VirtualChannelSelector; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; +import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.partitioner.ConfigurableStreamPartitioner; +import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.streamstatus.StreamStatus; + +import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables; +import org.apache.flink.shaded.guava18.com.google.common.collect.Maps; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * {@link RecordDeserializer}-like interface for recovery. To avoid additional virtual method calls on the + * non-recovery hotpath, this interface is not extending RecordDeserializer. + */ +interface Demultiplexer extends AutoCloseable { + RecordDeserializer.DeserializationResult getNextRecord(DeserializationDelegate<StreamElement> deserializationDelegate) throws IOException; + + void setNextBuffer(Buffer buffer) throws IOException; + + void select(VirtualChannelSelector event); + + @Override + void close(); +} + +class NoDataDemultiplexer implements Demultiplexer { + private final InputChannelInfo channelInfo; + + public NoDataDemultiplexer(InputChannelInfo channelInfo) { + this.channelInfo = channelInfo; + } + + @Override + public RecordDeserializer.DeserializationResult getNextRecord(DeserializationDelegate<StreamElement> deserializationDelegate) { + throw getException(); + } + + @Override + public void setNextBuffer(Buffer buffer) { + throw getException(); + } + + @Override + public void select(VirtualChannelSelector event) { + throw getException(); + } + + private IllegalStateException getException() { + return new IllegalStateException(channelInfo + " should not receive any data/events during recovery"); + } + + @Override + public void close() { + } +} + +/** + * Parameter structure to pass all relevant information to the factory methods of @{@link Demultiplexer}. + */ +class DemultiplexParameters { + final IOManager ioManager; + final InflightDataRescalingDescriptor channelMapping; + final Function<Integer, StreamPartitioner<?>> gatePartitionerRetriever; + final SerializationDelegate<StreamRecord> delegate; + final int numberOfChannels; + final int subtaskIndex; + + @SuppressWarnings("unchecked") + DemultiplexParameters( + TypeSerializer<?> inputSerializer, + IOManager ioManager, + InflightDataRescalingDescriptor channelMapping, + Function<Integer, StreamPartitioner<?>> gatePartitionerRetriever, + int numberOfChannels, + int subtaskIndex) { + delegate = new SerializationDelegate<>((TypeSerializer<StreamRecord>) inputSerializer); + this.ioManager = ioManager; + this.channelMapping = channelMapping; + this.gatePartitionerRetriever = gatePartitionerRetriever; + this.numberOfChannels = numberOfChannels; + this.subtaskIndex = subtaskIndex; + } +} + +/** + * Demultiplexes buffers on subtask-level. + * + * <p>Example: If the current task has been downscaled from 2 to 1. Then the only new subtask needs to handle data + * originating from old subtasks 0 and 1. In this case, {@link #demultiplexersForSubtasks} contains + * {@code 0->ChannelDemultiplexer0, 1->ChannelDemultiplexer1}. + * + * <p>Since this the outer demultiplexing layer, it is also responsible for summarizing watermark and stream + * statuses of the (nested) virtual channels. + */ +class SubtaskDemultiplexer implements Demultiplexer { + private final Map<Integer, ChannelDemultiplexer> demultiplexersForSubtasks; + + /** Keep track of the last emitted watermark for all (nested) virtual channels. */ + private final Map<VirtualChannelSelector, Watermark> lastWatermarks; + + /** Keep track of the last emitted stream status for all (nested) virtual channels. */ + private final Map<VirtualChannelSelector, StreamStatus> streamStatuses; + + private VirtualChannelSelector currentSelector; + + private ChannelDemultiplexer selectedSubtask; + + public SubtaskDemultiplexer(Map<Integer, ChannelDemultiplexer> demultiplexersForSubtasks, int totalChannels) { + this.demultiplexersForSubtasks = demultiplexersForSubtasks; + final Map.Entry<Integer, ChannelDemultiplexer> defaultSelection = + Iterables.get(demultiplexersForSubtasks.entrySet(), 0); + selectedSubtask = defaultSelection.getValue(); + currentSelector = new VirtualChannelSelector(defaultSelection.getKey(), + selectedSubtask.selectedChannelIndex); + + // initialize watermarks and streamStatuses for all nested virtual channels + this.lastWatermarks = Maps.newHashMapWithExpectedSize(totalChannels); + this.streamStatuses = Maps.newHashMapWithExpectedSize(totalChannels); + getChannelSelectors().forEach(selector -> { + lastWatermarks.put(selector, Watermark.UNINITIALIZED); + streamStatuses.put(selector, StreamStatus.ACTIVE); + }); + } + + public Stream<VirtualChannelSelector> getChannelSelectors() { + return demultiplexersForSubtasks.values().stream().flatMap(ChannelDemultiplexer::getChannelSelectors); Review comment: I inlined it in ctor. It's using an instance field and is called from `SubtaskDemultplexer`, so I don't see a good way to make it static. ---------------------------------------------------------------- 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: [email protected]
