pnowojski commented on a change in pull request #16774: URL: https://github.com/apache/flink/pull/16774#discussion_r688299759
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/FinishedOperatorChain.java ########## @@ -0,0 +1,367 @@ +/* + * 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.tasks; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.runtime.checkpoint.CheckpointMetaData; +import org.apache.flink.runtime.checkpoint.CheckpointOptions; +import org.apache.flink.runtime.checkpoint.channel.ChannelStateWriter; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.io.network.api.writer.RecordWriter; +import org.apache.flink.runtime.io.network.api.writer.RecordWriterDelegate; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.plugable.SerializationDelegate; +import org.apache.flink.runtime.state.CheckpointStreamFactory; +import org.apache.flink.runtime.state.TaskStateManager; +import org.apache.flink.streaming.api.graph.StreamConfig; +import org.apache.flink.streaming.api.graph.StreamEdge; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; +import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorFactoryUtil; +import org.apache.flink.streaming.api.operators.StreamTaskStateInitializer; +import org.apache.flink.streaming.runtime.io.RecordWriterOutput; +import org.apache.flink.streaming.runtime.io.StreamTaskSourceInput; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxExecutorFactory; +import org.apache.flink.util.OutputTag; +import org.apache.flink.util.SerializedValue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.Supplier; + +/** + * The {@link OperatorChain} that is used for restoring tasks that are {@link + * TaskStateManager#isFinishedOnRestore()}. + */ +@Internal +public class FinishedOperatorChain<OUT, OP extends StreamOperator<OUT>> + extends OperatorChain<OUT, OP> { + + public FinishedOperatorChain( + StreamTask<OUT, OP> containingTask, + RecordWriterDelegate<SerializationDelegate<StreamRecord<OUT>>> recordWriterDelegate) { + super(containingTask, recordWriterDelegate); + } + + @Override + public boolean isFinishedOnRestore() { + return true; + } + + @Override + public void dispatchOperatorEvent(OperatorID operator, SerializedValue<OperatorEvent> event) { + throw new UnsupportedOperationException(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) {} + + @Override + public void endInput(int inputId) throws Exception {} + + @Override + public void initializeStateAndOpenOperators( + StreamTaskStateInitializer streamTaskStateInitializer) {} + + @Override + public void finishOperators(StreamTaskActionExecutor actionExecutor) throws Exception {} + + @Override + public void close() throws IOException { + super.close(); + } + + // ------------------------------------------------------------------------ Review comment: Ops 😳 good catch :) ########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/tasks/OperatorChain.java ########## @@ -161,7 +164,11 @@ public OperatorChain( Map<StreamEdge, RecordWriterOutput<?>> streamOutputMap = new HashMap<>(outEdgesInOrder.size()); this.streamOutputs = new RecordWriterOutput<?>[outEdgesInOrder.size()]; - this.finishedOnRestore = finishedOnRestore; + this.finishedOnRestoreInput = + this.isFinishedOnRestore() Review comment: I think that would require a more extensive refactor and extracting the logic of creating operators/outputs from `OperatorChain` to some builder class. The problem is that currently constructor of `OperatorChain()` needs to call `getFinishedOnRestoreInputOrDefault()`. So the field `finishedOnRestoreInput` can not be initialised in the `FinishedOperatorChain`, as it would be too late. So we are stuck with: ``` protected final @Nullable FinishedOnRestoreInput finishedOnRestoreInput; (...) this.finishedOnRestoreInput = this.isFinishedOnRestore() ? new FinishedOnRestoreInput( streamOutputs, configuration.getInputs(userCodeClassloader).length) : null; ``` being defined and initialised in the `OperatorChain`. With that in mind, I think it would be actually a slightly less readable code if we had the getter to that `@Nullable` field in the `FinishedOperatorChain` and `RegularOperatorChain`. As I wrote in the beginning, probably the right way to go would be to construct all outputs/operators in a builder class, and only inject them to the `OperatorChain`. That way field `finishedOnRestoreInput` could be defined in the `FinishedOperatorChain` only. But I would prefer to leave it for a future improvement. Or am I missing something? -- 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]
