Skip to content

Commit

Permalink
[Feature][starter] support user define parameter on spark/flink engine
Browse files Browse the repository at this point in the history
  • Loading branch information
liunaijie committed Feb 26, 2024
1 parent 71854ba commit d999cac
Show file tree
Hide file tree
Showing 8 changed files with 282 additions and 13 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -29,8 +29,11 @@

import java.nio.file.Path;
import java.nio.file.Paths;
import java.util.List;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.stream.Collectors;

/** Used to build the {@link Config} from config file. */
@Slf4j
Expand All @@ -43,13 +46,25 @@ private ConfigBuilder() {
// utility class and cannot be instantiated
}

private static Config ofInner(@NonNull Path filePath) {
private static Config ofInner(@NonNull Path filePath, List<String> variables) {
Config config =
ConfigFactory.parseFile(filePath.toFile())
.resolve(ConfigResolveOptions.defaults().setAllowUnresolved(true))
.resolveWith(
ConfigFactory.systemProperties(),
ConfigResolveOptions.defaults().setAllowUnresolved(true));
if (variables != null) {
Map<String, String> map =
variables.stream()
.filter(Objects::nonNull)
.map(variable -> variable.split("=", 2))
.filter(pair -> pair.length == 2F)
.collect(Collectors.toMap(pair -> pair[0], pair -> pair[1]));
config =
config.resolveWith(
ConfigFactory.parseMap(map),
ConfigResolveOptions.defaults().setAllowUnresolved(true));
}
return ConfigShadeUtils.decryptConfig(config);
}

Expand All @@ -58,13 +73,22 @@ public static Config of(@NonNull String filePath) {
return of(path);
}

public static Config of(@NonNull String filePath, List<String> variables) {
Path path = Paths.get(filePath);
return of(path, variables);
}

public static Config of(@NonNull Path filePath) {
return of(filePath, null);
}

public static Config of(@NonNull Path filePath, List<String> variables) {
log.info("Loading config file from path: {}", filePath);
Optional<ConfigAdapter> adapterSupplier = ConfigAdapterUtils.selectAdapter(filePath);
Config config =
adapterSupplier
.map(adapter -> of(adapter, filePath))
.orElseGet(() -> ofInner(filePath));
.map(adapter -> of(adapter, filePath, variables))
.orElseGet(() -> ofInner(filePath, variables));
log.info("Parsed config file: \n{}", config.root().render(CONFIG_RENDER_OPTIONS));
return config;
}
Expand All @@ -88,17 +112,30 @@ public static Config of(@NonNull Map<String, Object> objectMap, boolean isEncryp
return config;
}

public static Config of(@NonNull ConfigAdapter configAdapter, @NonNull Path filePath) {
public static Config of(
@NonNull ConfigAdapter configAdapter, @NonNull Path filePath, List<String> variables) {
log.info("With config adapter spi {}", configAdapter.getClass().getName());
try {
Map<String, Object> flattenedMap = configAdapter.loadConfig(filePath);
Config config = ConfigFactory.parseMap(flattenedMap);
if (variables != null) {
Map<String, String> map =
variables.stream()
.filter(Objects::nonNull)
.map(variable -> variable.split("=", 2))
.filter(pair -> pair.length == 2F)
.collect(Collectors.toMap(pair -> pair[0], pair -> pair[1]));
config =
config.resolveWith(
ConfigFactory.parseMap(map),
ConfigResolveOptions.defaults().setAllowUnresolved(true));
}
return ConfigShadeUtils.decryptConfig(config);
} catch (Exception warn) {
log.warn(
"Loading config failed with spi {}, fallback to HOCON loader.",
configAdapter.getClass().getName());
return ofInner(filePath);
return ofInner(filePath, variables);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public FlinkTaskExecuteCommand(FlinkCommandArgs flinkCommandArgs) {
public void execute() throws CommandExecuteException {
Path configFile = FileUtils.getConfigPath(flinkCommandArgs);
checkConfigExist(configFile);
Config config = ConfigBuilder.of(configFile);
Config config = ConfigBuilder.of(configFile, flinkCommandArgs.getVariables());
// if user specified job name using command line arguments, override config option
if (!flinkCommandArgs.getJobName().equals(Constants.LOGO)) {
config =
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ public SparkTaskExecuteCommand(SparkCommandArgs sparkCommandArgs) {
public void execute() throws CommandExecuteException {
Path configFile = FileUtils.getConfigPath(sparkCommandArgs);
checkConfigExist(configFile);
Config config = ConfigBuilder.of(configFile);
Config config = ConfigBuilder.of(configFile, sparkCommandArgs.getVariables());
if (!sparkCommandArgs.getJobName().equals(Constants.LOGO)) {
config =
config.withValue(
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,52 @@
/*
* 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.seatunnel.example.flink.v2;

import org.apache.seatunnel.core.starter.SeaTunnel;
import org.apache.seatunnel.core.starter.exception.CommandException;
import org.apache.seatunnel.core.starter.flink.args.FlinkCommandArgs;

import java.io.FileNotFoundException;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.file.Paths;
import java.util.Collections;

public class SeaTunnelFlinkVariablesExample {

public static void main(String[] args)
throws FileNotFoundException, URISyntaxException, CommandException {
String configurePath =
args.length > 0 ? args[0] : "/examples/fake_to_console.variables.conf";
String configFile = getTestConfigFile(configurePath);
FlinkCommandArgs flinkCommandArgs = new FlinkCommandArgs();
flinkCommandArgs.setConfigFile(configFile);
flinkCommandArgs.setCheckConfig(false);
flinkCommandArgs.setVariables(Collections.singletonList("nameVal=abc"));
SeaTunnel.run(flinkCommandArgs.buildCommand());
}

public static String getTestConfigFile(String configFile)
throws FileNotFoundException, URISyntaxException {
URL resource = SeaTunnelApiExample.class.getResource(configFile);
if (resource == null) {
throw new FileNotFoundException("Can't find config file: " + configFile);
}
return Paths.get(resource.toURI()).toString();
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,61 @@
#
# 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.
#
######
###### This config file is a demonstration of streaming processing in seatunnel config
######

env {
job.mode = "BATCH"
parallelism = 2
}

source {
# This is a example source plugin **only for test and demonstrate the feature source plugin**
FakeSource {
result_table_name = "fake"
row.num = 16
string.template = ["abc", "def"]
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 {

# 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-v2
sql {
source_table_name = "fake"
query = "select * from fake where name = '"${nameVal}"' "
result_table_name = "sql"
}
}

sink {
Console {
}

# 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
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,16 +17,36 @@

package org.apache.seatunnel.example.spark.v2;

import org.apache.seatunnel.common.config.DeployMode;
import org.apache.seatunnel.core.starter.SeaTunnel;
import org.apache.seatunnel.core.starter.exception.CommandException;
import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs;

import java.io.FileNotFoundException;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.file.Paths;

public class SeaTunnelApiExample {

public static void main(String[] args)
throws FileNotFoundException, URISyntaxException, CommandException {
String configurePath = args.length > 0 ? args[0] : "/examples/spark.batch.conf";
ExampleUtils.builder(configurePath);
String configFile = getTestConfigFile(configurePath);
SparkCommandArgs sparkCommandArgs = new SparkCommandArgs();
sparkCommandArgs.setConfigFile(configFile);
sparkCommandArgs.setCheckConfig(false);
sparkCommandArgs.setVariables(null);
sparkCommandArgs.setDeployMode(DeployMode.CLIENT);
SeaTunnel.run(sparkCommandArgs.buildCommand());
}

private static String getTestConfigFile(String configFile)
throws FileNotFoundException, URISyntaxException {
URL resource = SeaTunnelApiExample.class.getResource(configFile);
if (resource == null) {
throw new FileNotFoundException("Can't find config file: " + configFile);
}
return Paths.get(resource.toURI()).toString();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,23 +19,23 @@

import org.apache.seatunnel.common.config.DeployMode;
import org.apache.seatunnel.core.starter.SeaTunnel;
import org.apache.seatunnel.core.starter.exception.CommandException;
import org.apache.seatunnel.core.starter.spark.args.SparkCommandArgs;

import java.io.FileNotFoundException;
import java.net.URISyntaxException;
import java.net.URL;
import java.nio.file.Paths;
import java.util.Collections;

public class ExampleUtils {
public class SeaTunnelSparkVariablesExample {

public static void builder(String configurePath)
throws FileNotFoundException, URISyntaxException, CommandException {
public static void main(String[] args) throws FileNotFoundException, URISyntaxException {
String configurePath = args.length > 0 ? args[0] : "/examples/spark.batch.variables.conf";
String configFile = getTestConfigFile(configurePath);
SparkCommandArgs sparkCommandArgs = new SparkCommandArgs();
sparkCommandArgs.setConfigFile(configFile);
sparkCommandArgs.setCheckConfig(false);
sparkCommandArgs.setVariables(null);
sparkCommandArgs.setVariables(Collections.singletonList("intVal=1"));
sparkCommandArgs.setDeployMode(DeployMode.CLIENT);
SeaTunnel.run(sparkCommandArgs.buildCommand());
}
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,99 @@
#
# 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.
#

######
###### This config file is a demonstration of batch processing in SeaTunnel config
######

env {
# You can set spark configuration here
# 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 = 1
spark.executor.cores = 1
spark.executor.memory = "1g"
spark.master = local
}

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>"
c_array = "array<int>"
c_string = string
c_boolean = boolean
c_tinyint = tinyint
c_smallint = smallint
c_int = int
c_bigint = bigint
c_float = float
c_double = double
c_decimal = "decimal(30, 8)"
c_null = "null"
c_bytes = bytes
c_date = date
c_timestamp = timestamp
}
}
result_table_name = "fake"
}

# You can also use other input plugins, such as hdfs
# hdfs {
# result_table_name = "accesslog"
# path = "hdfs://hadoop-cluster-01/nginx/accesslog"
# format = "json"
# }

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

transform {
# split data by specific delimiter

# you can also use other transform plugins, such as sql
sql {
source_table_name = "fake"
query = "select c_map,c_array,c_string,c_boolean,c_tinyint,c_smallint,c_int,c_bigint,c_float,c_double,c_null,c_bytes,c_date,c_timestamp from fake where c_int = "${intVal}" "
result_table_name = "sql"
}

# 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-v2
}

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

# you can also you other output plugins, such as sql
# hdfs {
# path = "hdfs://hadoop-cluster-01/nginx/accesslog_processed"
# save_mode = "append"
# }

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

0 comments on commit d999cac

Please sign in to comment.