pnowojski commented on code in PR #27783: URL: https://github.com/apache/flink/pull/27783#discussion_r2999390852
########## flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/channel/ChannelStateFilteringHandler.java: ########## @@ -0,0 +1,418 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.runtime.checkpoint.InflightDataRescalingDescriptor; +import org.apache.flink.runtime.checkpoint.RescaleMappings; +import org.apache.flink.runtime.io.network.api.SubtaskConnectionDescriptor; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer; +import org.apache.flink.runtime.io.network.api.serialization.RecordDeserializer.DeserializationResult; +import org.apache.flink.runtime.io.network.api.serialization.SpillingAdaptiveSpanningRecordDeserializer; +import org.apache.flink.runtime.io.network.buffer.Buffer; +import org.apache.flink.runtime.io.network.partition.consumer.InputGate; +import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.streaming.runtime.io.recovery.RecordFilter; +import org.apache.flink.streaming.runtime.io.recovery.RecordFilterContext; +import org.apache.flink.streaming.runtime.io.recovery.VirtualChannel; +import org.apache.flink.streaming.runtime.io.recovery.VirtualChannelRecordFilterFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.flink.util.Preconditions.checkNotNull; + +/** + * Filters recovered channel state buffers during the channel-state-unspilling phase, removing + * records that do not belong to the current subtask after rescaling. + * + * <p>Uses a per-gate architecture: each {@link InputGate} gets its own {@link GateFilterHandler} + * with the correct serializer, so multi-input tasks (e.g., TwoInputStreamTask) correctly + * deserialize different record types on different gates. + */ +@Internal +public class ChannelStateFilteringHandler { + + /** + * Handles record filtering for a single input gate. Each gate has its own serializer and set of + * virtual channels, allowing different gates to handle different record types independently. + */ + static class GateFilterHandler<T> { + + private final Map<SubtaskConnectionDescriptor, VirtualChannel<T>> virtualChannels; + private final StreamElementSerializer<T> serializer; + private final DeserializationDelegate<StreamElement> deserializationDelegate; + private final DataOutputSerializer outputSerializer; + private final byte[] lengthBuffer = new byte[4]; + + GateFilterHandler( + Map<SubtaskConnectionDescriptor, VirtualChannel<T>> virtualChannels, + StreamElementSerializer<T> serializer) { + this.virtualChannels = checkNotNull(virtualChannels); + this.serializer = checkNotNull(serializer); + this.deserializationDelegate = new NonReusingDeserializationDelegate<>(serializer); + this.outputSerializer = new DataOutputSerializer(128); + } + + /** + * Deserializes records from {@code sourceBuffer}, applies the virtual channel's record + * filter, and re-serializes the surviving records into new buffers. + */ + List<Buffer> filterAndRewrite( + int oldSubtaskIndex, + int oldChannelIndex, + Buffer sourceBuffer, + BufferSupplier bufferSupplier) + throws IOException, InterruptedException { + + SubtaskConnectionDescriptor key = + new SubtaskConnectionDescriptor(oldSubtaskIndex, oldChannelIndex); + VirtualChannel<T> vc = virtualChannels.get(key); + if (vc == null) { + throw new IllegalStateException( + "No VirtualChannel found for key: " + + key + + "; known channels are " + + virtualChannels.keySet()); + } + + vc.setNextBuffer(sourceBuffer); + + List<StreamElement> filteredElements = new ArrayList<>(); + + while (true) { + DeserializationResult result = vc.getNextRecord(deserializationDelegate); + if (result.isFullRecord()) { + filteredElements.add(deserializationDelegate.getInstance()); + } + if (result.isBufferConsumed()) { + break; + } + } + + return serializeToBuffers(filteredElements, bufferSupplier); + } + + /** + * Serializes stream elements into buffers using the length-prefixed format (4-byte + * big-endian length + record bytes) expected by Flink's record deserializers. + */ + private List<Buffer> serializeToBuffers( + List<StreamElement> elements, BufferSupplier bufferSupplier) + throws IOException, InterruptedException { + + List<Buffer> resultBuffers = new ArrayList<>(); + + if (elements.isEmpty()) { + return resultBuffers; + } + + Buffer currentBuffer = bufferSupplier.requestBufferBlocking(); + + for (StreamElement element : elements) { + outputSerializer.clear(); + serializer.serialize(element, outputSerializer); + int recordLength = outputSerializer.length(); + + writeLengthToBuffer(recordLength); + currentBuffer = + writeDataToBuffer( + lengthBuffer, 0, 4, currentBuffer, resultBuffers, bufferSupplier); + + byte[] serializedData = outputSerializer.getSharedBuffer(); + currentBuffer = + writeDataToBuffer( + serializedData, + 0, + recordLength, + currentBuffer, + resultBuffers, + bufferSupplier); + } + + if (currentBuffer.readableBytes() > 0) { + resultBuffers.add(currentBuffer.retainBuffer()); + } + currentBuffer.recycleBuffer(); + + return resultBuffers; + } + + private void writeLengthToBuffer(int length) { + lengthBuffer[0] = (byte) (length >> 24); + lengthBuffer[1] = (byte) (length >> 16); + lengthBuffer[2] = (byte) (length >> 8); + lengthBuffer[3] = (byte) length; + } + + /** + * Writes data to the current buffer, spilling into new buffers from {@code bufferSupplier} + * when the current one is full. + * + * @return the buffer to continue writing into (may differ from the input buffer). + */ + private Buffer writeDataToBuffer( + byte[] data, + int dataOffset, + int dataLength, + Buffer currentBuffer, + List<Buffer> resultBuffers, + BufferSupplier bufferSupplier) + throws IOException, InterruptedException { + int offset = dataOffset; + int remaining = dataLength; + + while (remaining > 0) { + int writableBytes = currentBuffer.getMaxCapacity() - currentBuffer.getSize(); + + if (writableBytes == 0) { + if (currentBuffer.readableBytes() > 0) { + resultBuffers.add(currentBuffer.retainBuffer()); + } + currentBuffer.recycleBuffer(); + currentBuffer = bufferSupplier.requestBufferBlocking(); Review Comment: I think it would be better to squash that commit here, to avoid merging broken code given that we already have some working fix for 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. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
