Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[Imporve][Seatunnel-code][Seatunnel-flink-starter] customize operator parallel for flink and spark #2941

Merged
merged 13 commits into from
Oct 18, 2022
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
10 changes: 10 additions & 0 deletions docs/en/connector-v2/sink/common-options.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,18 +5,27 @@
| name | type | required | default value |
| ----------------- | ------ | -------- | ------------- |
| source_table_name | string | no | - |
| parallelism | int | no | - |


### source_table_name [string]

When `source_table_name` is not specified, the current plug-in processes the data set `dataset` output by the previous plugin in the configuration file;

When `source_table_name` is specified, the current plug-in is processing the data set corresponding to this parameter.

### parallelism [int]

When `parallelism` is not specified, the `parallelism` in env is used by default.

When parallelism is specified, it will override the parallelism in env.

## Examples

```bash
source {
FakeSourceStream {
parallelism = 2
result_table_name = "fake"
field_name = "name,age"
}
Expand All @@ -37,6 +46,7 @@ transform {

sink {
console {
parallelism = 3
source_table_name = "fake_name"
}
}
Expand Down
7 changes: 7 additions & 0 deletions docs/en/connector-v2/source/common-options.md
Original file line number Diff line number Diff line change
Expand Up @@ -5,13 +5,20 @@
| name | type | required | default value |
| ----------------- | ------ | -------- | ------------- |
| result_table_name | string | no | - |
| parallelism | int | no | - |

### result_table_name [string]

When `result_table_name` is not specified, the data processed by this plugin will not be registered as a data set `(dataStream/dataset)` that can be directly accessed by other plugins, or called a temporary table `(table)` ;

When `result_table_name` is specified, the data processed by this plugin will be registered as a data set `(dataStream/dataset)` that can be directly accessed by other plugins, or called a temporary table `(table)` . The data set `(dataStream/dataset)` registered here can be directly accessed by other plugins by specifying `source_table_name` .

### parallelism [int]

When `parallelism` is not specified, the `parallelism` in env is used by default.

When parallelism is specified, it will override the parallelism in env.

## Example

```bash
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,8 +31,6 @@ public final class Constants {

public static final String SOURCE_SERIALIZATION = "source.serialization";

public static final String SOURCE_PARALLELISM = "parallelism";

public static final String HDFS_ROOT = "hdfs.root";

public static final String HDFS_USER = "hdfs.user";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.constants.CollectionConstants;
import org.apache.seatunnel.core.starter.exception.TaskExecuteException;
import org.apache.seatunnel.flink.FlinkEnvironment;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
Expand All @@ -32,6 +33,7 @@

import com.google.common.collect.Lists;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.streaming.api.datastream.DataStreamSink;
import org.apache.flink.types.Row;

import java.net.URL;
Expand Down Expand Up @@ -76,7 +78,11 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
SeaTunnelSink<SeaTunnelRow, Serializable, Serializable, Serializable> seaTunnelSink = plugins.get(i);
DataStream<Row> stream = fromSourceTable(sinkConfig).orElse(input);
seaTunnelSink.setTypeInfo((SeaTunnelRowType) TypeConverterUtils.convert(stream.getType()));
stream.sinkTo(new FlinkSink<>(seaTunnelSink)).name(seaTunnelSink.getPluginName());
DataStreamSink<Row> dataStreamSink = stream.sinkTo(new FlinkSink<>(seaTunnelSink)).name(seaTunnelSink.getPluginName());
liugddx marked this conversation as resolved.
Show resolved Hide resolved
if (sinkConfig.hasPath(CollectionConstants.PARALLELISM)) {
int parallelism = sinkConfig.getInt(CollectionConstants.PARALLELISM);
dataStreamSink.setParallelism(parallelism);
}
}
// the sink is the last stream
return null;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.source.SupportCoordinate;
import org.apache.seatunnel.common.constants.CollectionConstants;
import org.apache.seatunnel.common.constants.JobMode;
import org.apache.seatunnel.flink.FlinkEnvironment;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
Expand Down Expand Up @@ -75,6 +76,10 @@ public List<DataStream<Row>> execute(List<DataStream<Row>> upstreamDataStreams)
"SeaTunnel " + internalSource.getClass().getSimpleName(),
internalSource.getBoundedness() == org.apache.seatunnel.api.source.Boundedness.BOUNDED);
Config pluginConfig = pluginConfigs.get(i);
if (pluginConfig.hasPath(CollectionConstants.PARALLELISM)) {
int parallelism = pluginConfig.getInt(CollectionConstants.PARALLELISM);
sourceStream.setParallelism(parallelism);
}
registerResultTable(pluginConfig, sourceStream);
sources.add(sourceStream);
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.sink.SeaTunnelSink;
import org.apache.seatunnel.api.table.type.SeaTunnelRowType;
import org.apache.seatunnel.common.constants.CollectionConstants;
import org.apache.seatunnel.core.starter.exception.TaskExecuteException;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSinkPluginDiscovery;
Expand Down Expand Up @@ -71,6 +72,13 @@ public List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) throws
Config sinkConfig = pluginConfigs.get(i);
SeaTunnelSink<?, ?, ?, ?> seaTunnelSink = plugins.get(i);
Dataset<Row> dataset = fromSourceTable(sinkConfig, sparkEnvironment).orElse(input);
int parallelism;
if (sinkConfig.hasPath(CollectionConstants.PARALLELISM)) {
parallelism = sinkConfig.getInt(CollectionConstants.PARALLELISM);
} else {
parallelism = sparkEnvironment.getSparkConf().getInt(CollectionConstants.PARALLELISM, 1);
}
dataset.sparkSession().read().option(CollectionConstants.PARALLELISM, parallelism);
// TODO modify checkpoint location
seaTunnelSink.setTypeInfo((SeaTunnelRowType) TypeConverterUtils.convert(dataset.schema()));
SparkSinkInjector.inject(dataset.write(), seaTunnelSink).option("checkpointLocation", "/tmp").save();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.api.common.JobContext;
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.constants.CollectionConstants;
import org.apache.seatunnel.common.utils.SerializationUtils;
import org.apache.seatunnel.plugin.discovery.PluginIdentifier;
import org.apache.seatunnel.plugin.discovery.seatunnel.SeaTunnelSourcePluginDiscovery;
Expand Down Expand Up @@ -56,15 +57,15 @@ public List<Dataset<Row>> execute(List<Dataset<Row>> upstreamDataStreams) {
SeaTunnelSource<?, ?, ?> source = plugins.get(i);
Config pluginConfig = pluginConfigs.get(i);
int parallelism;
if (pluginConfig.hasPath(Constants.SOURCE_PARALLELISM)) {
parallelism = pluginConfig.getInt(Constants.SOURCE_PARALLELISM);
if (pluginConfig.hasPath(CollectionConstants.PARALLELISM)) {
parallelism = pluginConfig.getInt(CollectionConstants.PARALLELISM);
} else {
parallelism = sparkEnvironment.getSparkConf().getInt(Constants.SOURCE_PARALLELISM, 1);
parallelism = sparkEnvironment.getSparkConf().getInt(CollectionConstants.PARALLELISM, 1);
}
Dataset<Row> dataset = sparkEnvironment.getSparkSession()
.read()
.format(SeaTunnelSource.class.getSimpleName())
.option(Constants.SOURCE_PARALLELISM, parallelism)
.option(CollectionConstants.PARALLELISM, parallelism)
.option(Constants.SOURCE_SERIALIZATION, SerializationUtils.objectToString(source))
.schema((StructType) TypeConverterUtils.convert(source.getProducedType())).load();
sources.add(dataset);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,32 +28,35 @@ env {

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
FakeSource {
result_table_name = "fake"
row.num = 16
schema = {
fields {
name = "string"
age = "int"
}
FakeSource {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Please add parallelism config in FakeSource. Also please update doc

parallelism = 2
result_table_name = "fake"
row.num = 16
schema = {
fields {
name = "string"
age = "int"
}
}
}

# If you would like to get more information about how to configure seatunnel and see full list of source plugins,
# please go to https://seatunnel.apache.org/docs/category/source-v2
}

transform {
sql {
sql = "select name,age from fake"
}
sql {
sql = "select name,age from fake"
}

# If you would like to get more information about how to configure seatunnel and see full list of transform plugins,
# please go to https://seatunnel.apache.org/docs/category/transform
}

sink {
Console {}
Console {
parallelism = 3
}

# If you would like to get more information about how to configure seatunnel and see full list of sink plugins,
# please go to https://seatunnel.apache.org/docs/category/sink-v2
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,7 @@ env {
# see available properties defined by spark: https://spark.apache.org/docs/latest/configuration.html#available-properties
#job.mode = BATCH
job.name = "SeaTunnel"
spark.executor.instances = 2
spark.executor.instances = 1
spark.executor.cores = 1
spark.executor.memory = "1g"
spark.master = local
Expand All @@ -34,6 +34,7 @@ source {
# This is a example input plugin **only for test and demonstrate the feature input plugin**
FakeSource {
row.num = 16
parallelism = 2
schema = {
fields {
c_map = "map<string, string>"
Expand Down Expand Up @@ -82,7 +83,9 @@ transform {

sink {
# choose stdout output plugin to output data to console
Console {}
Console {
parallelism = 2
}

# you can also you other output plugins, such as sql
# hdfs {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
import org.apache.seatunnel.api.source.SeaTunnelSource;
import org.apache.seatunnel.api.table.type.SeaTunnelRow;
import org.apache.seatunnel.common.Constants;
import org.apache.seatunnel.common.constants.CollectionConstants;
import org.apache.seatunnel.common.utils.SerializationUtils;
import org.apache.seatunnel.translation.spark.source.batch.BatchSourceReader;
import org.apache.seatunnel.translation.spark.source.micro.MicroBatchSourceReader;
Expand Down Expand Up @@ -59,14 +60,14 @@ public DataSourceReader createReader(StructType rowType, DataSourceOptions optio
@Override
public DataSourceReader createReader(DataSourceOptions options) {
SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(options);
int parallelism = options.getInt(Constants.SOURCE_PARALLELISM, 1);
int parallelism = options.getInt(CollectionConstants.PARALLELISM, 1);
return new BatchSourceReader(seaTunnelSource, parallelism);
}

@Override
public MicroBatchReader createMicroBatchReader(Optional<StructType> rowTypeOptional, String checkpointLocation, DataSourceOptions options) {
SeaTunnelSource<SeaTunnelRow, ?, ?> seaTunnelSource = getSeaTunnelSource(options);
Integer parallelism = options.getInt(Constants.SOURCE_PARALLELISM, 1);
Integer parallelism = options.getInt(CollectionConstants.PARALLELISM, 1);
Integer checkpointInterval = options.getInt(Constants.CHECKPOINT_INTERVAL, CHECKPOINT_INTERVAL_DEFAULT);
String checkpointPath = StringUtils.replacePattern(checkpointLocation, "sources/\\d+", "sources-state");
Configuration configuration = SparkSession.getActiveSession().get().sparkContext().hadoopConfiguration();
Expand Down