kl0u commented on a change in pull request #13824: URL: https://github.com/apache/flink/pull/13824#discussion_r514960481
########## File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java ########## @@ -0,0 +1,293 @@ +/* + * 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.translators; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink.Sink; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.api.java.typeutils.TypeExtractor; +import org.apache.flink.streaming.api.graph.StreamGraph; +import org.apache.flink.streaming.api.graph.TransformationTranslator; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperatorFactory; +import org.apache.flink.streaming.api.transformations.SinkTransformation; +import org.apache.flink.streaming.runtime.operators.sink.BatchCommitterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.BatchGlobalCommitterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.GlobalStreamingCommitterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.StatefulWriterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.StatelessWriterOperatorFactory; +import org.apache.flink.streaming.runtime.operators.sink.StreamingCommitterOperatorFactory; +import org.apache.flink.streaming.util.graph.StreamGraphUtils; +import org.apache.flink.streaming.util.typeutils.CommittableTypeInformation; + +import java.lang.reflect.Type; +import java.util.Collection; +import java.util.Collections; + +import static org.apache.flink.api.java.typeutils.TypeExtractionUtils.typeToClass; +import static org.apache.flink.util.Preconditions.checkNotNull; +import static org.apache.flink.util.Preconditions.checkState; + +/** + * A {@link TransformationTranslator} for the {@link SinkTransformation}. + */ +@Internal +public class SinkTransformationTranslator<InputT, CommT, WriterStateT, GlobalCommT> implements + TransformationTranslator<Object, SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT>> { + + @Override + public Collection<Integer> translateForBatch( + SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> transformation, + Context context) { + + StreamGraphUtils.validateTransformationUid(context.getStreamGraph(), transformation); + final int parallelism = getParallelism(transformation, context); + + int writerId = addWriter( + transformation, + parallelism, context); + int committerId = addCommitter( + writerId, + transformation, + new BatchCommitterOperatorFactory<>(transformation.getSink()), + parallelism, context); + addGlobalCommitter( + committerId >= 0 ? committerId : writerId, + transformation, + new BatchGlobalCommitterOperatorFactory<>(transformation.getSink()), context); + return Collections.emptyList(); + } + + @Override + public Collection<Integer> translateForStreaming( + SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> transformation, + Context context) { + + StreamGraphUtils.validateTransformationUid(context.getStreamGraph(), transformation); + + final int parallelism = getParallelism(transformation, context); + + int writerId = addWriter( + transformation, + parallelism, context); + int committerId = addCommitter( + writerId, + transformation, + new StreamingCommitterOperatorFactory<>(transformation.getSink()), + parallelism, context); + addGlobalCommitter( + committerId >= 0 ? committerId : writerId, + transformation, + new GlobalStreamingCommitterOperatorFactory<>(transformation.getSink()), context); + + return Collections.emptyList(); + } + + /** + * Add a sink writer node to the stream graph. + * + * @param sinkTransformation The transformation that the writer belongs to + * @param parallelism The parallelism of the writer + * + * @return The stream node id of the writer + */ + private int addWriter( + SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation, + int parallelism, + Context context) { + final boolean hasState = sinkTransformation + .getSink() + .getWriterStateSerializer() + .isPresent(); + checkState(sinkTransformation.getInputs().size() == 1); + @SuppressWarnings("unchecked") + final Transformation<InputT> input = (Transformation<InputT>) sinkTransformation + .getInputs() + .get(0); + final TypeInformation<InputT> inputTypeInfo = input.getOutputType(); + + final StreamOperatorFactory<CommT> writer = + hasState ? new StatefulWriterOperatorFactory<>(sinkTransformation.getSink()) : new StatelessWriterOperatorFactory<>( + sinkTransformation.getSink()); + + final int writerId = addOperatorToStreamGraph( + writer, input.getId(), + inputTypeInfo, + extractCommittableTypeInformation(sinkTransformation.getSink()), Review comment: In general in the translation process here we make an implicit assumption that if we do not have a `Committer`, then we also do not have a `GlobalCommitter`, right? Because when we set the output type of the `Writer` we always set it to `CommT`. In addition, when we extract the output type of the writer in the `extractCommittableTypeInformation()` we only look if there is a `committableSerializer`. If this is not present then the `Writer` cannot communicate with the `GlobalCommitter`. Is this something we want? And if yes, then we should have a validation step for the sink that throws an exception if we have a sink with a `GlobalCommitter` but without a `Committer`. If we do not want that, then I think we should adjust the `addWrier`. Is this analysis correct @guoweiM ? ---------------------------------------------------------------- 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]
