gaoyunhaii commented on a change in pull request #13824:
URL: https://github.com/apache/flink/pull/13824#discussion_r514888912
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStreamSink.java
##########
@@ -33,19 +36,33 @@
@Public
public class DataStreamSink<T> {
- private final LegacySinkTransformation<T> transformation;
+ private final PhysicalTransformation<T> transformation;
@SuppressWarnings("unchecked")
protected DataStreamSink(DataStream<T> inputStream, StreamSink<T>
operator) {
- this.transformation = new
LegacySinkTransformation<T>(inputStream.getTransformation(), "Unnamed",
operator, inputStream.getExecutionEnvironment().getParallelism());
+ this.transformation = (PhysicalTransformation<T>) new
LegacySinkTransformation<>(inputStream.getTransformation(), "Unnamed",
operator, inputStream.getExecutionEnvironment().getParallelism());
Review comment:
nit: too long line
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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(),
Review comment:
nit: put `input.getId()` in new line, and also other calls to
`addOperatorToStreamGraph`.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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()),
+ "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
+ * @param parallelism The parallelism of the committer
+ *
+ * @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,
+ int parallelism,
+ Context context) {
+
+ if
(!sinkTransformation.getSink().createCommitter().isPresent()) {
+ return -1;
+ }
+
+ final CommittableTypeInformation<CommT> committableTypeInfo =
extractCommittableTypeInformation(
+ sinkTransformation.getSink());
+ checkNotNull(committableTypeInfo);
+
+ 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().createGlobalCommitter().isPresent()) {
+ return;
+ }
+
+ addOperatorToStreamGraph(
+ globalCommitterFactory, inputId,
+
checkNotNull(extractCommittableTypeInformation(sinkTransformation.getSink())),
+ 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()));
+ }
+ streamGraph.addEdge(inputId, transformationId, 0);
+
+ return transformationId;
+ }
+
+ private CommittableTypeInformation<CommT>
extractCommittableTypeInformation(Sink<InputT, CommT, WriterStateT,
GlobalCommT> sink) {
Review comment:
We might extract the type information only one time at the begin of the
translation
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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);
Review comment:
nit: put `context` in new line, and also for line 72 and 76
##########
File path:
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamSinkITCase.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.IntegerTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.streaming.util.FiniteTestSource;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+
+/**
+ * Integration test for {@link org.apache.flink.api.connector.sink.Sink} run
time implementation.
+ */
+public class StreamSinkITCase extends AbstractTestBase {
Review comment:
Would it be better to use "SinkITCase" since it tests both streaming and
batch mode ?
##########
File path:
flink-streaming-java/src/test/java/org/apache/flink/streaming/api/graph/SinkTransformationTranslatorTest.java
##########
@@ -0,0 +1,248 @@
+/*
+ * 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.graph;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeutils.base.IntSerializer;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.core.io.SimpleVersionedSerializerTypeSerializerProxy;
+import org.apache.flink.streaming.api.datastream.DataStreamSink;
+import org.apache.flink.streaming.api.datastream.DataStreamSource;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.api.operators.ChainingStrategy;
+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.StatelessWriterOperatorFactory;
+import
org.apache.flink.streaming.runtime.operators.sink.StreamingCommitterOperatorFactory;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.util.TestLogger;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.util.Arrays;
+import java.util.Collection;
+
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.CoreMatchers.instanceOf;
+import static org.hamcrest.MatcherAssert.assertThat;
+
+/**
+ * Tests for {@link
org.apache.flink.streaming.api.transformations.SinkTransformation}.
+ */
+@RunWith(Parameterized.class)
+public class SinkTransformationTranslatorTest extends TestLogger {
+
+ @Parameterized.Parameters
Review comment:
might also set name to reveal the parameter values, like
`@Parameterized.Parameters(name="mode: {0}, ....")`
##########
File path:
flink-tests/src/test/java/org/apache/flink/test/streaming/runtime/StreamSinkITCase.java
##########
@@ -0,0 +1,119 @@
+/*
+ * 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.test.streaming.runtime;
+
+import org.apache.flink.api.common.RuntimeExecutionMode;
+import org.apache.flink.api.common.typeinfo.IntegerTypeInfo;
+import org.apache.flink.api.java.tuple.Tuple3;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.configuration.ExecutionOptions;
+import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
+import org.apache.flink.streaming.runtime.operators.sink.TestSink;
+import org.apache.flink.streaming.util.FiniteTestSource;
+import org.apache.flink.test.util.AbstractTestBase;
+
+import org.junit.Test;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import static java.util.stream.Collectors.joining;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.hamcrest.Matchers.containsInAnyOrder;
+
+/**
+ * Integration test for {@link org.apache.flink.api.connector.sink.Sink} run
time implementation.
+ */
+public class StreamSinkITCase extends AbstractTestBase {
+
+ static final List<Integer> SOURCE_DATA = Arrays.asList(
+ 895, 127, 148, 161, 148, 662, 822, 491, 275, 122,
+ 850, 630, 682, 765, 434, 970, 714, 795, 288, 422);
+
+ static final Queue<String> STREAMING_COMMIT_QUEUE = new
ConcurrentLinkedQueue<>();
+
+ static final Queue<String> STREAMING_GLOBAL_COMMIT_QUEUE = new
ConcurrentLinkedQueue<>();
+
+ static final Queue<String> BATCH_COMMIT_QUEUE = new
ConcurrentLinkedQueue<>();
+
+ static final Queue<String> BATCH_GLOBAL_COMMIT_QUEUE = new
ConcurrentLinkedQueue<>();
+
+ @Test
+ public void streamingExecutionMode() throws Exception {
+ StreamExecutionEnvironment env =
StreamExecutionEnvironment.getExecutionEnvironment();
+ env.enableCheckpointing(100);
+ FiniteTestSource<Integer> source = new
FiniteTestSource<>(SOURCE_DATA);
+
+ env.addSource(source, IntegerTypeInfo.INT_TYPE_INFO)
+ .addSink(TestSink
+ .newBuilder()
+
.setDefaultCommitter((Supplier<Queue<String>> & Serializable) () ->
STREAMING_COMMIT_QUEUE)
+
.setGlobalCommitter((Supplier<Queue<String>> & Serializable) () ->
STREAMING_GLOBAL_COMMIT_QUEUE)
+ .build());
+
+ env.execute();
+
+ final List<String> expectedCommittedData = SOURCE_DATA
+ .stream()
+ .map(x -> Tuple3.of(x, null,
Long.MIN_VALUE).toString())
+ .collect(
+ Collectors.toList());
+
+ // source send data two times
+ expectedCommittedData.addAll(expectedCommittedData);
Review comment:
Might remove this line since add itself might contains risks. We may
direct use `.flatMap(x -> Stream.of(x, x))` to duplicate the elements in the
streaming operator before.
##########
File path:
flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/translators/SinkTransformationTranslator.java
##########
@@ -0,0 +1,292 @@
+/*
+ * 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()),
+ "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
+ * @param parallelism The parallelism of the committer
+ *
+ * @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,
+ int parallelism,
+ Context context) {
+
+ if
(!sinkTransformation.getSink().createCommitter().isPresent()) {
+ return -1;
+ }
+
+ final CommittableTypeInformation<CommT> committableTypeInfo =
extractCommittableTypeInformation(
+ sinkTransformation.getSink());
+ checkNotNull(committableTypeInfo);
+
+ 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().createGlobalCommitter().isPresent()) {
+ return;
+ }
+
+ addOperatorToStreamGraph(
+ globalCommitterFactory, inputId,
+
checkNotNull(extractCommittableTypeInformation(sinkTransformation.getSink())),
+ 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()));
+ }
+ streamGraph.addEdge(inputId, transformationId, 0);
+
+ return transformationId;
+ }
+
+ private CommittableTypeInformation<CommT>
extractCommittableTypeInformation(Sink<InputT, CommT, WriterStateT,
GlobalCommT> sink) {
+ if (sink.getCommittableSerializer().isPresent()) {
+ final Type committableType =
TypeExtractor.getParameterType(
+ Sink.class,
+ sink.getClass(),
+ 1);
+ return new CommittableTypeInformation<>(
+ typeToClass(committableType), () ->
sink.getCommittableSerializer().get());
Review comment:
nit: ` () -> sink.getCommittableSerializer().get())` in new line
----------------------------------------------------------------
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]