You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2020/10/29 12:23:24 UTC

[GitHub] [flink] kl0u commented on a change in pull request #13824: [FLINK-19736] Add the SinkTransformation

kl0u commented on a change in pull request #13824:
URL: https://github.com/apache/flink/pull/13824#discussion_r514211286



##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/transformations/SinkTransformation.java
##########
@@ -0,0 +1,91 @@
+/*
+ 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.api.transformations;
+
+import org.apache.flink.annotation.Internal;
+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.operators.ChainingStrategy;
+
+import org.apache.flink.shaded.guava18.com.google.common.collect.Lists;
+
+import java.util.Collections;
+import java.util.List;
+
+import static org.apache.flink.util.Preconditions.checkNotNull;
+
+/**
+ * This Transformation represents a {@link org.apache.flink.api.connector.sink.Sink}.
+ *
+ * @param <InputT> The input type of the {@link org.apache.flink.api.connector.sink.Writer}
+ * @param <CommT> The committable type of the {@link org.apache.flink.api.connector.sink.Writer}
+ * @param <WriterStateT> The state type of the {@link org.apache.flink.api.connector.sink.Writer}
+ * @param <GlobalCommT> The global committable type of the {@link org.apache.flink.api.connector.sink.GlobalCommitter}
+ */
+@Internal
+public class SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> extends PhysicalTransformation<Object> {
+
+	private final Transformation<InputT> input;
+
+	private final Sink<InputT, CommT, WriterStateT, GlobalCommT> sink;
+
+	private ChainingStrategy chainingStrategy;
+
+	public SinkTransformation(
+			Transformation<InputT> input,
+			Sink<InputT, CommT, WriterStateT, GlobalCommT> sink,
+			String name,
+			int parallelism) {
+		super(name, TypeExtractor.getForClass(Object.class), parallelism);
+		this.input = checkNotNull(input);
+		this.sink = checkNotNull(sink);
+	}
+
+	@Override
+	public void setChainingStrategy(ChainingStrategy strategy) {
+		chainingStrategy = strategy;

Review comment:
       We can add a `checkNotNull` here and if we also set a default value to `ALWAYS` (as in the `AbstractStreamOperatorFactory`) then we could remove the `null` check in the `SinkTransformationTranslator.addOperatorToStreamGraph()#257`.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.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.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 java.util.Collection;
+import java.util.Collections;
+
+/**
+ * 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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new BatchCommitterOperatorFactory<>(transformation.getSink()),
+				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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new StreamingCommitterOperatorFactory<>(transformation.getSink()),
+				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
+	 *
+	 * @return The stream node id of the writer
+	 */
+	private int addWriter(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final boolean hasState = sinkTransformation
+				.getSink()
+				.getWriterStateSerializer()
+				.isPresent();
+		final TypeInformation<CommT> outTypeInfo = TypeExtractor.createTypeInfo(

Review comment:
       I guess this will change after https://github.com/apache/flink/pull/13825 but we could even move it to a separate method as it is used in many places.

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.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.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 java.util.Collection;
+import java.util.Collections;
+
+/**
+ * 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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new BatchCommitterOperatorFactory<>(transformation.getSink()),
+				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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new StreamingCommitterOperatorFactory<>(transformation.getSink()),
+				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
+	 *
+	 * @return The stream node id of the writer
+	 */
+	private int addWriter(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final boolean hasState = sinkTransformation
+				.getSink()
+				.getWriterStateSerializer()
+				.isPresent();
+		final TypeInformation<CommT> outTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+		@SuppressWarnings("unchecked")
+		final Transformation<InputT> input = (Transformation<InputT>) sinkTransformation
+				.getInputs()
+				.get(0);
+		final TypeInformation<InputT> inputTypeInfo = input.getOutputType();
+
+		final int parallelism = getParallelism(sinkTransformation, context);
+
+		final StreamOperatorFactory<CommT> writer =
+				hasState ? new StatefulWriterOperatorFactory<>(sinkTransformation.getSink()) : new StatelessWriterOperatorFactory<>(
+						sinkTransformation.getSink());
+
+		final int writerId = addOperatorToStreamGraph(
+				writer, input.getId(),
+				inputTypeInfo,
+				outTypeInfo,
+				"Sink Writer:",
+				parallelism,
+				sinkTransformation.getMaxParallelism(),
+				sinkTransformation,
+				context);
+
+		StreamGraphUtils.configureResourceProperties(
+				context.getStreamGraph(),
+				writerId,
+				sinkTransformation);
+		return writerId;
+	}
+
+	/**
+	 * Try to add a sink committer to the stream graph.
+	 *
+	 * @param inputId The committer's input stream node id
+	 * @param sinkTransformation The transformation that the committer belongs to
+	 * @param committerFactory The committer operator's factory
+	 *
+	 * @return The stream node id of the committer or -1 if the sink topology does not include a committer.
+	 */
+	private int addCommitter(
+			int inputId,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			OneInputStreamOperatorFactory<CommT, CommT> committerFactory,
+			Context context) {
+
+		if (!sinkTransformation.getSink().getCommittableSerializer().isPresent()) {
+			return -1;
+		}
+
+		final TypeInformation<CommT> committableTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+
+		final int parallelism = getParallelism(sinkTransformation, context);
+
+		return addOperatorToStreamGraph(
+				committerFactory, inputId,
+				committableTypeInfo,
+				committableTypeInfo,
+				"Sink Committer:",
+				parallelism,
+				sinkTransformation.getMaxParallelism(),
+				sinkTransformation,
+				context);
+	}
+
+	/**
+	 * Try to add a sink global committer to the stream graph.
+	 *
+	 * @param inputId The global committer's input stream node id.
+	 * @param sinkTransformation The transformation that the global committer belongs to
+	 * @param globalCommitterFactory The global committer factory
+	 */
+	private void addGlobalCommitter(
+			int inputId,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			OneInputStreamOperatorFactory<CommT, GlobalCommT> globalCommitterFactory,
+			Context context) {
+
+		if (!sinkTransformation.getSink().getGlobalCommittableSerializer().isPresent()) {
+			return;
+		}
+
+		final TypeInformation<CommT> committableTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+
+		addOperatorToStreamGraph(
+				globalCommitterFactory, inputId,
+				committableTypeInfo,
+				null,
+				"Sink Global Committer:",
+				1,
+				1,
+				sinkTransformation,
+				context);
+	}
+
+	private int getParallelism(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		return sinkTransformation.getParallelism() != ExecutionConfig.PARALLELISM_DEFAULT
+				? sinkTransformation.getParallelism()
+				: context.getStreamGraph().getExecutionConfig().getParallelism();
+	}
+
+	/**
+	 * Add a operator to the {@link StreamGraph}.
+	 *
+	 * @param operatorFactory The operator factory
+	 * @param inputId The upstream stream node id of the operator
+	 * @param inTypeInfo The input type information of the operator
+	 * @param outTypInfo The output type information of the operator
+	 * @param prefix The prefix of the name and uid of the operator
+	 * @param parallelism The parallelism of the operator
+	 * @param maxParallelism The max parallelism of the operator
+	 * @param sinkTransformation The sink transformation which the operator belongs to
+	 *
+	 * @return The stream node id of the operator
+	 */
+	private <IN, OUT> int addOperatorToStreamGraph(
+			StreamOperatorFactory<OUT> operatorFactory, int inputId,
+			TypeInformation<IN> inTypeInfo,
+			TypeInformation<OUT> outTypInfo,
+			String prefix,
+			int parallelism,
+			int maxParallelism,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final StreamGraph streamGraph = context.getStreamGraph();
+		final String slotSharingGroup = context.getSlotSharingGroup();
+		final int transformationId = Transformation.getNewNodeId();
+
+		final ChainingStrategy chainingStrategy = sinkTransformation.getChainingStrategy();
+		if (chainingStrategy != null) {
+			operatorFactory.setChainingStrategy(chainingStrategy);
+		}
+
+		streamGraph.addOperator(
+				transformationId,
+				slotSharingGroup,
+				sinkTransformation.getCoLocationGroupKey(),
+				operatorFactory,
+				inTypeInfo,
+				outTypInfo,
+				String.format("%s %s", prefix, sinkTransformation.getName()));
+
+		streamGraph.setParallelism(transformationId, parallelism);
+		streamGraph.setMaxParallelism(transformationId, maxParallelism);
+
+		StreamGraphUtils.configureBufferTimeout(
+				streamGraph,
+				transformationId,
+				sinkTransformation,
+				context.getDefaultBufferTimeout());
+		if (sinkTransformation.getUid() != null) {
+			streamGraph.setTransformationUID(
+					transformationId,
+					String.format("%s %s", prefix, sinkTransformation.getUid()));
+		}

Review comment:
       Do we know what happens if nothing is set?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.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.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 java.util.Collection;
+import java.util.Collections;
+
+/**
+ * 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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new BatchCommitterOperatorFactory<>(transformation.getSink()),
+				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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new StreamingCommitterOperatorFactory<>(transformation.getSink()),
+				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
+	 *
+	 * @return The stream node id of the writer
+	 */
+	private int addWriter(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final boolean hasState = sinkTransformation
+				.getSink()
+				.getWriterStateSerializer()
+				.isPresent();
+		final TypeInformation<CommT> outTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+		@SuppressWarnings("unchecked")
+		final Transformation<InputT> input = (Transformation<InputT>) sinkTransformation
+				.getInputs()
+				.get(0);
+		final TypeInformation<InputT> inputTypeInfo = input.getOutputType();
+
+		final int parallelism = getParallelism(sinkTransformation, context);
+
+		final StreamOperatorFactory<CommT> writer =
+				hasState ? new StatefulWriterOperatorFactory<>(sinkTransformation.getSink()) : new StatelessWriterOperatorFactory<>(
+						sinkTransformation.getSink());
+
+		final int writerId = addOperatorToStreamGraph(
+				writer, input.getId(),
+				inputTypeInfo,
+				outTypeInfo,
+				"Sink Writer:",
+				parallelism,
+				sinkTransformation.getMaxParallelism(),
+				sinkTransformation,
+				context);
+
+		StreamGraphUtils.configureResourceProperties(
+				context.getStreamGraph(),
+				writerId,
+				sinkTransformation);
+		return writerId;
+	}
+
+	/**
+	 * Try to add a sink committer to the stream graph.
+	 *
+	 * @param inputId The committer's input stream node id
+	 * @param sinkTransformation The transformation that the committer belongs to
+	 * @param committerFactory The committer operator's factory
+	 *
+	 * @return The stream node id of the committer or -1 if the sink topology does not include a committer.
+	 */
+	private int addCommitter(
+			int inputId,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			OneInputStreamOperatorFactory<CommT, CommT> committerFactory,
+			Context context) {
+
+		if (!sinkTransformation.getSink().getCommittableSerializer().isPresent()) {
+			return -1;
+		}
+
+		final TypeInformation<CommT> committableTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+
+		final int parallelism = getParallelism(sinkTransformation, context);

Review comment:
       Are we sure that in both `BATCH` and `STREAMING` the parallelism of the `Committter` will be equal to `parallelism`? Why not passing it as an argument and figuring the actual value in the `translateForStreaming` or `translateForBatch`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.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.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 java.util.Collection;
+import java.util.Collections;
+
+/**
+ * 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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new BatchCommitterOperatorFactory<>(transformation.getSink()),
+				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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new StreamingCommitterOperatorFactory<>(transformation.getSink()),
+				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
+	 *
+	 * @return The stream node id of the writer
+	 */
+	private int addWriter(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final boolean hasState = sinkTransformation
+				.getSink()
+				.getWriterStateSerializer()
+				.isPresent();
+		final TypeInformation<CommT> outTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+		@SuppressWarnings("unchecked")
+		final Transformation<InputT> input = (Transformation<InputT>) sinkTransformation
+				.getInputs()
+				.get(0);

Review comment:
       Here we need to verify if the size exactly 1 before calling `get()`. Probably a `checkState()`?

##########
File path: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.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.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 java.util.Collection;
+import java.util.Collections;
+
+/**
+ * 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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new BatchCommitterOperatorFactory<>(transformation.getSink()),
+				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);
+
+		int writerId = addWriter(transformation, context);
+		int committerId = addCommitter(
+				writerId,
+				transformation,
+				new StreamingCommitterOperatorFactory<>(transformation.getSink()),
+				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
+	 *
+	 * @return The stream node id of the writer
+	 */
+	private int addWriter(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final boolean hasState = sinkTransformation
+				.getSink()
+				.getWriterStateSerializer()
+				.isPresent();
+		final TypeInformation<CommT> outTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+		@SuppressWarnings("unchecked")
+		final Transformation<InputT> input = (Transformation<InputT>) sinkTransformation
+				.getInputs()
+				.get(0);
+		final TypeInformation<InputT> inputTypeInfo = input.getOutputType();
+
+		final int parallelism = getParallelism(sinkTransformation, context);
+
+		final StreamOperatorFactory<CommT> writer =
+				hasState ? new StatefulWriterOperatorFactory<>(sinkTransformation.getSink()) : new StatelessWriterOperatorFactory<>(
+						sinkTransformation.getSink());
+
+		final int writerId = addOperatorToStreamGraph(
+				writer, input.getId(),
+				inputTypeInfo,
+				outTypeInfo,
+				"Sink Writer:",
+				parallelism,
+				sinkTransformation.getMaxParallelism(),
+				sinkTransformation,
+				context);
+
+		StreamGraphUtils.configureResourceProperties(
+				context.getStreamGraph(),
+				writerId,
+				sinkTransformation);
+		return writerId;
+	}
+
+	/**
+	 * Try to add a sink committer to the stream graph.
+	 *
+	 * @param inputId The committer's input stream node id
+	 * @param sinkTransformation The transformation that the committer belongs to
+	 * @param committerFactory The committer operator's factory
+	 *
+	 * @return The stream node id of the committer or -1 if the sink topology does not include a committer.
+	 */
+	private int addCommitter(
+			int inputId,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			OneInputStreamOperatorFactory<CommT, CommT> committerFactory,
+			Context context) {
+
+		if (!sinkTransformation.getSink().getCommittableSerializer().isPresent()) {
+			return -1;
+		}
+
+		final TypeInformation<CommT> committableTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+
+		final int parallelism = getParallelism(sinkTransformation, context);
+
+		return addOperatorToStreamGraph(
+				committerFactory, inputId,
+				committableTypeInfo,
+				committableTypeInfo,
+				"Sink Committer:",
+				parallelism,
+				sinkTransformation.getMaxParallelism(),
+				sinkTransformation,
+				context);
+	}
+
+	/**
+	 * Try to add a sink global committer to the stream graph.
+	 *
+	 * @param inputId The global committer's input stream node id.
+	 * @param sinkTransformation The transformation that the global committer belongs to
+	 * @param globalCommitterFactory The global committer factory
+	 */
+	private void addGlobalCommitter(
+			int inputId,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			OneInputStreamOperatorFactory<CommT, GlobalCommT> globalCommitterFactory,
+			Context context) {
+
+		if (!sinkTransformation.getSink().getGlobalCommittableSerializer().isPresent()) {
+			return;
+		}
+
+		final TypeInformation<CommT> committableTypeInfo = TypeExtractor.createTypeInfo(
+				Sink.class,
+				sinkTransformation.getSink().getClass(),
+				1,
+				null,
+				null);
+
+		addOperatorToStreamGraph(
+				globalCommitterFactory, inputId,
+				committableTypeInfo,
+				null,
+				"Sink Global Committer:",
+				1,
+				1,
+				sinkTransformation,
+				context);
+	}
+
+	private int getParallelism(
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		return sinkTransformation.getParallelism() != ExecutionConfig.PARALLELISM_DEFAULT
+				? sinkTransformation.getParallelism()
+				: context.getStreamGraph().getExecutionConfig().getParallelism();
+	}
+
+	/**
+	 * Add a operator to the {@link StreamGraph}.
+	 *
+	 * @param operatorFactory The operator factory
+	 * @param inputId The upstream stream node id of the operator
+	 * @param inTypeInfo The input type information of the operator
+	 * @param outTypInfo The output type information of the operator
+	 * @param prefix The prefix of the name and uid of the operator
+	 * @param parallelism The parallelism of the operator
+	 * @param maxParallelism The max parallelism of the operator
+	 * @param sinkTransformation The sink transformation which the operator belongs to
+	 *
+	 * @return The stream node id of the operator
+	 */
+	private <IN, OUT> int addOperatorToStreamGraph(
+			StreamOperatorFactory<OUT> operatorFactory, int inputId,
+			TypeInformation<IN> inTypeInfo,
+			TypeInformation<OUT> outTypInfo,
+			String prefix,
+			int parallelism,
+			int maxParallelism,
+			SinkTransformation<InputT, CommT, WriterStateT, GlobalCommT> sinkTransformation,
+			Context context) {
+		final StreamGraph streamGraph = context.getStreamGraph();
+		final String slotSharingGroup = context.getSlotSharingGroup();
+		final int transformationId = Transformation.getNewNodeId();
+

Review comment:
        Do we want this setting to apply it to all operators or only the `Writer`? I think that it seems more relevant for the writer only as that is what the user "sees", but I am not 100% sure.




----------------------------------------------------------------
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:
users@infra.apache.org