AHeise commented on a change in pull request #13845:
URL: https://github.com/apache/flink/pull/13845#discussion_r519226903



##########
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);
+       }
+
+       public void select(VirtualChannelSelector selector) {
+               currentSelector = selector;
+               selectedSubtask = 
demultiplexersForSubtasks.get(selector.getSubtaskIndex());
+               if (selectedSubtask == null) {
+                       throw new IllegalStateException(
+                               "Cannot select " + selector + "; known channels 
are " + getChannelSelectors().collect(Collectors.toList()));
+               }
+               selectedSubtask.select(selector);
+       }
+
+       @Override
+       public void setNextBuffer(Buffer buffer) throws IOException {
+               selectedSubtask.setNextBuffer(buffer);
+       }
+
+       @Override
+       public RecordDeserializer.DeserializationResult 
getNextRecord(DeserializationDelegate<StreamElement> deserializationDelegate) 
throws IOException {
+               do {
+                       RecordDeserializer.DeserializationResult result = 
selectedSubtask.getNextRecord(deserializationDelegate);
+
+                       // special handling of watermarks and stream status
+                       if (result.isFullRecord()) {
+                               final StreamElement element = 
deserializationDelegate.getInstance();
+                               if (element.isWatermark()) {
+                                       // basically, do not emit a watermark 
if not all virtual channel are past it
+                                       lastWatermarks.put(currentSelector, 
element.asWatermark());
+                                       final Watermark minWatermark = 
lastWatermarks.values().stream()
+                                               
.min(Comparator.comparing(Watermark::getTimestamp))
+                                               .orElseThrow(() -> new 
IllegalStateException("Should always have a min watermark"));
+                                       // at least one virtual channel has no 
watermark, so don't emit any watermark yet
+                                       if 
(minWatermark.equals(Watermark.UNINITIALIZED)) {
+                                               continue;
+                                       }
+                                       
deserializationDelegate.setInstance(minWatermark);
+                               } else if (element.isStreamStatus()) {
+                                       streamStatuses.put(currentSelector, 
element.asStreamStatus());
+                                       // summarize statuses across all 
virtual channels
+                                       // duplicate statuses are filtered in 
StatusWatermarkValve
+                                       if 
(streamStatuses.values().stream().anyMatch(s -> s.equals(StreamStatus.ACTIVE))) 
{
+                                               
deserializationDelegate.setInstance(StreamStatus.ACTIVE);
+                                       }
+                               }
+                       }
+
+                       return result;
+                       // loop is only re-executed for suppressed watermark
+               } while (true);
+       }
+
+       public void close() {
+               
demultiplexersForSubtasks.values().forEach(Demultiplexer::close);
+       }
+
+       static Demultiplexer forChannel(InputChannelInfo info, 
DemultiplexParameters parameters) {
+               final int[] oldSubtaskIndexes = 
parameters.channelMapping.getOldSubtaskIndexes(parameters.subtaskIndex);
+               if (oldSubtaskIndexes.length == 0) {
+                       return new NoDataDemultiplexer(info);
+               }
+               final int[] oldChannelIndexes = 
parameters.channelMapping.getChannelMapping(info.getGateIdx())
+                       .getOldChannelIndexes(info.getInputChannelIdx());
+               if (oldChannelIndexes.length == 0) {
+                       return new NoDataDemultiplexer(info);
+               }
+               int totalChannels = oldSubtaskIndexes.length * 
oldChannelIndexes.length;
+               Map<Integer, ChannelDemultiplexer> demultiplexersForSubtasks = 
Arrays.stream(oldSubtaskIndexes).boxed()
+                       .collect(Collectors.toMap(
+                               Function.identity(),
+                               oldSubtaskIndex -> 
ChannelDemultiplexer.forChannel(oldSubtaskIndex, info, parameters, 
totalChannels)
+                       ));
+               return new SubtaskDemultiplexer(demultiplexersForSubtasks, 
totalChannels);
+       }
+
+       @Override
+       public String toString() {
+               return "SubtaskDemultiplexer{" +
+                       "demultiplexersForSubtasks=" + 
demultiplexersForSubtasks +
+                       '}';
+       }
+}
+
+/**
+ * Demultiplexes buffers on channel-level.
+ *
+ * <p>Example: If the upstream task has been downscaled from 2 to 1. Then, old 
channels 0 and 1 are both
+ * processed over new channel 0. So this channel demultiplexer has two {@link 
#recordDeserializersForChannels} associated
+ * with the respective old channels.
+ *
+ * <p>For all non-unique mappings of new channels to old channels (see
+ * {@link org.apache.flink.runtime.io.network.api.writer.SubtaskStateMapper} 
for more details), a filter
+ * verifies if the restored record should be indeed processed by this subtask 
or if it should be filtered out and
+ * be processed at a different subtask.
+ */
+class ChannelDemultiplexer implements Demultiplexer {
+       private final Map<Integer, 
RecordDeserializer<DeserializationDelegate<StreamElement>>> 
recordDeserializersForChannels;
+
+       private static final Predicate<StreamRecord> NO_FILTER = record -> true;
+
+       private final Map<Integer, Predicate<StreamRecord>> filters;
+
+       private final int subtaskIndex;
+
+       @Nullable
+       private RecordDeserializer<DeserializationDelegate<StreamElement>> 
selectedChannel;
+
+       int selectedChannelIndex;
+
+       ChannelDemultiplexer(
+                       int subtaskIndex,
+                       Map<Integer, Predicate<StreamRecord>> 
oldChannelsWithFilters,
+                       DemultiplexParameters parameters,
+                       int totalChannels) {
+               this.subtaskIndex = subtaskIndex;
+               this.filters = oldChannelsWithFilters;
+               recordDeserializersForChannels = 
Maps.newHashMapWithExpectedSize(oldChannelsWithFilters.size());
+               for (final Integer oldChannel : 
oldChannelsWithFilters.keySet()) {
+                       recordDeserializersForChannels.put(oldChannel,
+                               new 
SpillingAdaptiveSpanningRecordDeserializer<>(parameters.ioManager.getSpillingDirectoriesPaths(),
+                                       
SpillingAdaptiveSpanningRecordDeserializer.DEFAULT_THRESHOLD_FOR_SPILLING / 
totalChannels,
+                                       
SpillingAdaptiveSpanningRecordDeserializer.DEFAULT_FILE_BUFFER_SIZE / 
totalChannels));
+               }
+
+               
recordDeserializersForChannels.entrySet().stream().findFirst().ifPresent(firstEntry
 -> {
+                       selectedChannel = firstEntry.getValue();
+                       selectedChannelIndex = firstEntry.getKey();
+               });
+       }
+
+       public Stream<VirtualChannelSelector> getChannelSelectors() {
+               return recordDeserializersForChannels.keySet().stream()
+                       .map(channelIndex -> new 
VirtualChannelSelector(subtaskIndex, channelIndex));
+       }
+
+       @Override
+       public RecordDeserializer.DeserializationResult 
getNextRecord(DeserializationDelegate<StreamElement> deserializationDelegate) 
throws IOException {
+               do {
+                       final RecordDeserializer.DeserializationResult result = 
selectedChannel.getNextRecord(deserializationDelegate);
+
+                       if (result.isBufferConsumed()) {
+                               
selectedChannel.getCurrentBuffer().recycleBuffer();
+                       }
+                       if (result.isFullRecord()) {
+                               final StreamElement element = 
deserializationDelegate.getInstance();
+                               if (element.isRecord() && 
!filters.get(selectedChannelIndex).test(element.asRecord())) {
+                                       continue;
+                               }
+                       }
+
+                       return result;
+                       // loop is re-executed for filtered full records.
+               } while (true);
+       }
+
+       public void select(VirtualChannelSelector selector) {
+               selectedChannelIndex = selector.getChannelIndex();
+               selectedChannel = 
recordDeserializersForChannels.get(selectedChannelIndex);
+               if (selectedChannel == null) {
+                       throw new IllegalStateException(
+                               "Cannot select " + selector + "; known channels 
are " + getChannelSelectors().collect(Collectors.toList()));
+               }
+       }
+
+       @Override
+       public void setNextBuffer(Buffer buffer) throws IOException {
+               selectedChannel.setNextBuffer(buffer);
+       }
+
+       public void close() {
+               for (RecordDeserializer<DeserializationDelegate<StreamElement>> 
deserializer :
+                       recordDeserializersForChannels.values()) {
+                       // recycle buffers and clear the deserializer.
+                       Buffer buffer = deserializer.getCurrentBuffer();
+                       if (buffer != null && !buffer.isRecycled()) {
+                               buffer.recycleBuffer();
+                       }
+                       deserializer.clear();
+               }
+       }
+
+       static ChannelDemultiplexer forChannel(
+                       int subtaskIndex,
+                       InputChannelInfo channelInfo,
+                       DemultiplexParameters parameters,
+                       int totalChannels) {
+               final InflightDataRescalingDescriptor mapping = 
parameters.channelMapping;
+               final RescaledChannelsMapping rescaledChannelsMapping =
+                       mapping.getChannelMapping(channelInfo.getGateIdx());
+               final int[] oldChannels = 
rescaledChannelsMapping.getOldChannelIndexes(channelInfo.getInputChannelIdx());
+
+               final Map<Integer, Predicate<StreamRecord>> 
oldChannelsWithFilters =
+                       Arrays.stream(oldChannels).boxed()
+                               .collect(Collectors.toMap(
+                                       Function.identity(),
+                                       oldChannel -> 
getFilterForChannel(channelInfo, parameters, rescaledChannelsMapping, 
oldChannel)));
+
+               return new ChannelDemultiplexer(
+                       subtaskIndex,
+                       oldChannelsWithFilters,
+                       parameters,
+                       totalChannels);
+       }
+
+       private static Predicate<StreamRecord> getFilterForChannel(
+                       InputChannelInfo channelInfo,
+                       DemultiplexParameters parameters,
+                       RescaledChannelsMapping rescaledChannelsMapping,
+                       Integer oldChannel) {
+               return 
rescaledChannelsMapping.getNewChannelIndexes(oldChannel).length <= 1 ?
+                       NO_FILTER :
+                       createFilter(channelInfo, parameters);
+       }
+
+       private static Predicate<StreamRecord> createFilter(InputChannelInfo 
channelInfo, DemultiplexParameters parameters) {
+               final StreamPartitioner partitioner = 
parameters.gatePartitionerRetriever.apply(channelInfo.getGateIdx());
+               final int inputChannelIdx = channelInfo.getInputChannelIdx();
+               final SerializationDelegate<StreamRecord> delegate = 
parameters.delegate;
+               partitioner.setup(parameters.numberOfChannels);
+               if (partitioner instanceof ConfigurableStreamPartitioner) {
+                       ((ConfigurableStreamPartitioner) 
partitioner).configure(KeyGroupRangeAssignment.UPPER_BOUND_MAX_PARALLELISM);
+               }
+               return streamRecord -> {
+                       delegate.setInstance(streamRecord);

Review comment:
       Since it's only used for the specific call inside the same thread it 
should be safe. However, it's cheap enough to create for each filter and it's 
probably much easier to reason about, so I changed it as you suggested.




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


Reply via email to