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



##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RescalingStreamTaskNetworkInput.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.io.InputStatus;
+import org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptor;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.VirtualChannelSelector;
+import 
org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import 
org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link StreamTaskNetworkInput} implementation that demultiplexes virtual 
channels.
+ *
+ * <p>The demultiplexing works in two dimensions for the following cases.
+ * <ul>
+ *     <li> Subtasks of the current operator have been collapsed in a 
round-robin fashion.
+ *     <li> The connected output operator has been rescaled (up and down!) and 
there is an overlap of channels (mostly
+ * relevant to keyed exchanges).
+ * </ul>
+ * In both cases, records from multiple old channels are received over one new 
physical channel, which need to
+ * demultiplex the record to correctly restore spanning records (similar to 
how StreamTaskNetworkInput works).
+ *
+ * <p>Note that when both cases occur at the same time (downscaling of several 
operators), there is the cross product of
+ * channels. So if two subtasks are collapsed and two channels overlap from 
the output side, there is a total of 4
+ * virtual channels.
+ */
+@Internal
+public final class RescalingStreamTaskNetworkInput<T> implements 
RecoverableStreamTaskInput<T> {

Review comment:
       The man motivation was to avoid a secondary implementations of 
`RecordDeserializer` as that would translate to virtual calls in the regular 
`StreamTaskNetworkInput`. I have not measured the impact but it was a major 
concern of @pnowojski .
   Now, it would be very well possible to subclass `StreamTaskNetworkInput` or 
extract a common super class. However, because `recordDeserializers` is of a 
different type without common ancestor (for CHA), it's hard to generalize.
   I had hoped for some input on how to solve it.

##########
File path: 
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/RescalingStreamTaskNetworkInput.java
##########
@@ -0,0 +1,283 @@
+/*
+ * 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.annotation.Internal;
+import org.apache.flink.api.common.typeutils.TypeSerializer;
+import org.apache.flink.core.io.InputStatus;
+import org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptor;
+import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter;
+import org.apache.flink.runtime.checkpoint.channel.InputChannelInfo;
+import org.apache.flink.runtime.event.AbstractEvent;
+import org.apache.flink.runtime.io.disk.iomanager.IOManager;
+import org.apache.flink.runtime.io.network.api.EndOfPartitionEvent;
+import org.apache.flink.runtime.io.network.api.VirtualChannelSelector;
+import 
org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult;
+import org.apache.flink.runtime.io.network.partition.consumer.BufferOrEvent;
+import 
org.apache.flink.runtime.io.network.partition.consumer.EndOfChannelStateEvent;
+import org.apache.flink.runtime.plugable.DeserializationDelegate;
+import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate;
+import org.apache.flink.streaming.runtime.partitioner.StreamPartitioner;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElement;
+import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer;
+import org.apache.flink.streaming.runtime.streamstatus.StatusWatermarkValve;
+
+import javax.annotation.Nullable;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Function;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+import static org.apache.flink.util.Preconditions.checkState;
+
+/**
+ * A {@link StreamTaskNetworkInput} implementation that demultiplexes virtual 
channels.
+ *
+ * <p>The demultiplexing works in two dimensions for the following cases.
+ * <ul>
+ *     <li> Subtasks of the current operator have been collapsed in a 
round-robin fashion.
+ *     <li> The connected output operator has been rescaled (up and down!) and 
there is an overlap of channels (mostly
+ * relevant to keyed exchanges).
+ * </ul>
+ * In both cases, records from multiple old channels are received over one new 
physical channel, which need to
+ * demultiplex the record to correctly restore spanning records (similar to 
how StreamTaskNetworkInput works).
+ *
+ * <p>Note that when both cases occur at the same time (downscaling of several 
operators), there is the cross product of
+ * channels. So if two subtasks are collapsed and two channels overlap from 
the output side, there is a total of 4
+ * virtual channels.
+ */
+@Internal
+public final class RescalingStreamTaskNetworkInput<T> implements 
RecoverableStreamTaskInput<T> {

Review comment:
       The main motivation was to avoid a secondary implementations of 
`RecordDeserializer` as that would translate to virtual calls in the regular 
`StreamTaskNetworkInput`. I have not measured the impact but it was a major 
concern of @pnowojski .
   Now, it would be very well possible to subclass `StreamTaskNetworkInput` or 
extract a common super class. However, because `recordDeserializers` is of a 
different type without common ancestor (for CHA), it's hard to generalize.
   I had hoped for some input on how to solve it.




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