pnowojski commented on a change in pull request #8476: [FLINK-12490][network] Introduce Input and NetworkInput interfaces URL: https://github.com/apache/flink/pull/8476#discussion_r287730119
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/io/NetworkInput.java ########## @@ -0,0 +1,154 @@ +/* + * 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.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.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.BufferOrEvent; +import org.apache.flink.runtime.plugable.DeserializationDelegate; +import org.apache.flink.runtime.plugable.NonReusingDeserializationDelegate; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamElementSerializer; + +import java.io.IOException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; + +import static org.apache.flink.util.Preconditions.checkState; + +/** + * Implementation of {@link Input} that wraps an input from network taken from {@link CheckpointBarrierHandler}. + */ +@Internal +public final class NetworkInput implements Input { + + private final CheckpointBarrierHandler barrierHandler; + + private final DeserializationDelegate<StreamElement> deserializationDelegate; + + private final RecordDeserializer<DeserializationDelegate<StreamElement>>[] recordDeserializers; + + private int currentChannel = UNSPECIFED; + + private RecordDeserializer<DeserializationDelegate<StreamElement>> currentRecordDeserializer = null; + + private boolean isFinished = false; + + @SuppressWarnings("unchecked") + public NetworkInput( + CheckpointBarrierHandler barrierHandler, + TypeSerializer<?> inputSerializer, + IOManager ioManager) { + this.barrierHandler = barrierHandler; + this.deserializationDelegate = new NonReusingDeserializationDelegate<>( + new StreamElementSerializer<>(inputSerializer)); + + // Initialize one deserializer per input channel + this.recordDeserializers = new SpillingAdaptiveSpanningRecordDeserializer[barrierHandler.getNumberOfInputChannels()]; + for (int i = 0; i < recordDeserializers.length; i++) { + recordDeserializers[i] = new SpillingAdaptiveSpanningRecordDeserializer<>( + ioManager.getSpillingDirectoriesPaths()); + } + } + + @Override + public StreamElement pollNext() throws Exception { + + while (true) { + // get the stream element from the deserializer + if (currentRecordDeserializer != null) { + DeserializationResult result = currentRecordDeserializer.getNextRecord(deserializationDelegate); + if (result.isBufferConsumed()) { + currentRecordDeserializer.getCurrentBuffer().recycleBuffer(); + currentRecordDeserializer = null; + } + + if (result.isFullRecord()) { + return deserializationDelegate.getInstance(); + } + } + + Optional<BufferOrEvent> bufferOrEvent = barrierHandler.pollNext(); + if (bufferOrEvent.isPresent()) { + processBufferOrEvent(bufferOrEvent.get()); + } else { + if (barrierHandler.isFinished()) { + isFinished = true; + checkState(barrierHandler.isAvailable().isDone(), "Finished BarrierHandler should be available"); + if (!barrierHandler.isEmpty()) { + throw new IllegalStateException("Trailing data in checkpoint barrier handler."); + } + } + return null; + } + } + } + + private void processBufferOrEvent(BufferOrEvent bufferOrEvent) throws IOException { + if (bufferOrEvent.isBuffer()) { + currentChannel = bufferOrEvent.getChannelIndex(); + currentRecordDeserializer = recordDeserializers[currentChannel]; + currentRecordDeserializer.setNextBuffer(bufferOrEvent.getBuffer()); + } + else { + // Event received + final AbstractEvent event = bufferOrEvent.getEvent(); + // TODO: with barrierHandler.isFinished() we might not need to support any events on this level. + if (event.getClass() != EndOfPartitionEvent.class) { + throw new IOException("Unexpected event: " + event); + } + } + } + + @Override + public int getLastChannel() { + return currentChannel; Review comment: Hmm, you are right. Somehow I felt that the the `lastChannel` is a better name for the user of the class while `currentChannel` better reflects the logic inside, but consistency trumps this. ---------------------------------------------------------------- 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] With regards, Apache Git Services
