Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -134,6 +134,10 @@ private void expand() {
CommittableMessageTypeInfo.noOutput(),
new SinkWriterOperatorFactory<>(sink)));
}

// Restore the previous parallelism of the environment before transforming
executionEnvironment.setParallelism(environmentParallelism);

final List<Transformation<?>> sinkTransformations =
executionEnvironment
.getTransformations()
Expand All @@ -147,8 +151,6 @@ private void expand() {
.getTransformations()
.remove(executionEnvironment.getTransformations().size() - 1);
}
// Restore the previous parallelism of the environment
executionEnvironment.setParallelism(environmentParallelism);
}

private <CommT> void addCommittingTopology(Sink<T> sink, DataStream<T> inputStream) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,6 +68,7 @@
import org.apache.flink.streaming.runtime.streamrecord.StreamRecord;
import org.apache.flink.streaming.runtime.tasks.StreamTask;
import org.apache.flink.streaming.util.NoOpIntMap;
import org.apache.flink.streaming.util.TestExpandingSink;
import org.apache.flink.util.Collector;
import org.apache.flink.util.TestLogger;

Expand Down Expand Up @@ -850,6 +851,28 @@ public void testResetBatchExchangeModeInStreamingExecution() {
.isEqualTo(StreamExchangeMode.UNDEFINED));
}

@Test
public void testAutoParallelismForExpandedTransformations() {
final StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment();

env.setParallelism(2);

DataStream<Integer> sourceDataStream = env.fromElements(1, 2, 3);
// Parallelism is set to -1 (default parallelism identifier) to imitate the behavior of
// the table planner. Parallelism should be set automatically after translating.
sourceDataStream.sinkTo(new TestExpandingSink()).setParallelism(-1);

StreamGraph graph = env.getStreamGraph();

graph.getStreamNodes()
.forEach(
node -> {
if (!node.getOperatorName().startsWith("Source")) {
assertEquals(2, node.getParallelism());
}
});
}

private static class FailingTransformation extends Transformation<String> {
private final int hashCode;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,71 @@
/*
* 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.util;

import org.apache.flink.api.connector.sink2.Committer;
import org.apache.flink.api.connector.sink2.Sink;
import org.apache.flink.core.io.SimpleVersionedSerializer;
import org.apache.flink.streaming.api.connector.sink2.CommittableMessage;
import org.apache.flink.streaming.api.connector.sink2.WithPostCommitTopology;
import org.apache.flink.streaming.api.connector.sink2.WithPreCommitTopology;
import org.apache.flink.streaming.api.connector.sink2.WithPreWriteTopology;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.functions.sink.DiscardingSink;

import java.io.IOException;

/** A test sink that expands into a simple subgraph. Do not use in runtime. */
public class TestExpandingSink
implements Sink<Integer>,
WithPreWriteTopology<Integer>,
WithPreCommitTopology<Integer, Integer>,
WithPostCommitTopology<Integer, Integer> {

@Override
public void addPostCommitTopology(DataStream<CommittableMessage<Integer>> committables) {
committables.addSink(new DiscardingSink<>());
}

@Override
public DataStream<CommittableMessage<Integer>> addPreCommitTopology(
DataStream<CommittableMessage<Integer>> committables) {
return committables.map(value -> value).returns(committables.getType());
}

@Override
public DataStream<Integer> addPreWriteTopology(DataStream<Integer> inputDataStream) {
return inputDataStream.map(new NoOpIntMap());
}

@Override
public PrecommittingSinkWriter<Integer, Integer> createWriter(InitContext context)
throws IOException {
return null;
}

@Override
public Committer<Integer> createCommitter() {
return null;
}

@Override
public SimpleVersionedSerializer<Integer> getCommittableSerializer() {
return null;
}
}