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

Feature flink config #486

Open
wants to merge 5 commits into
base: dev
Choose a base branch
from
Open
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 @@ -107,7 +107,7 @@ public class ConfigConstants {
public static final String OUTPUT_TABLE = "output_table";
public static final String TMP_TABLE = "tmp_table";
public static final String COLUMN_SEPARATOR = "column_separator";
public static final String LINE_SEPERATOR = "line_separator";
public static final String LINE_SEPARATOR = "line_separator";
public static final String DATA_DIR = "data_dir";

public static final String ENABLE_SPARK_HIVE_SUPPORT = "enable_spark_hive_support";
Expand Down Expand Up @@ -169,4 +169,8 @@ public class ConfigConstants {
public static final String SAVE_MODE = "save_mode";

public static final String UPSERT = "Upsert";

public static final String FILE_NAME = "file_name";

public static final String FLINK = "flink";
}
Original file line number Diff line number Diff line change
Expand Up @@ -43,6 +43,11 @@ public String getString(String key){
return String.valueOf(config.get(key));
}

public String getString(String key, String defaultValue) {
Object value = config.get(key);
return value != null ? String.valueOf(value) : defaultValue;
}

public List<String> getStringList(String key){
return (List<String>)config.get(key);
}
Expand All @@ -51,6 +56,11 @@ public Integer getInt(String key){
return Integer.valueOf(String.valueOf(config.get(key)));
}

public Integer getInt(String key, Integer defaultValue) {
Object value = config.get(key);
return value != null ? Integer.valueOf(String.valueOf(value)) : defaultValue;
}

public Boolean getBoolean(String key){
return Boolean.valueOf(String.valueOf(config.get(key)));
}
Expand All @@ -63,6 +73,11 @@ public Long getLong(String key){
return Long.valueOf(String.valueOf(config.get(key)));
}

public Long getLong(String key, Long defaultValue) {
Object value = config.get(key);
return value != null ? Long.valueOf(String.valueOf(value)) : defaultValue;
}

public Boolean has(String key) {
return config.get(key) != null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,13 +70,13 @@ public void init(Map<String, String> inputParameter, JobExecutionInfo jobExecuti
this.inputParameter.put(COLUMN_SEPARATOR,
errorDataParameterMap.get(CommonPropertyUtils.COLUMN_SEPARATOR) == null ?
CommonPropertyUtils.COLUMN_SEPARATOR_DEFAULT : errorDataParameterMap.get(CommonPropertyUtils.COLUMN_SEPARATOR));
this.inputParameter.put(LINE_SEPERATOR,
this.inputParameter.put(LINE_SEPARATOR,
errorDataParameterMap.get(CommonPropertyUtils.LINE_SEPARATOR) == null ?
CommonPropertyUtils.LINE_SEPARATOR_DEFAULT : errorDataParameterMap.get(CommonPropertyUtils.LINE_SEPARATOR));
} else {
this.inputParameter.put(ERROR_DATA_DIR, CommonPropertyUtils.getString(CommonPropertyUtils.ERROR_DATA_DIR, CommonPropertyUtils.ERROR_DATA_DIR_DEFAULT));
this.inputParameter.put(COLUMN_SEPARATOR, CommonPropertyUtils.getString(CommonPropertyUtils.COLUMN_SEPARATOR, CommonPropertyUtils.COLUMN_SEPARATOR_DEFAULT));
this.inputParameter.put(LINE_SEPERATOR, CommonPropertyUtils.getString(CommonPropertyUtils.LINE_SEPARATOR, CommonPropertyUtils.LINE_SEPARATOR_DEFAULT));
this.inputParameter.put(LINE_SEPARATOR, CommonPropertyUtils.getString(CommonPropertyUtils.LINE_SEPARATOR, CommonPropertyUtils.LINE_SEPARATOR_DEFAULT));
}

if (FILE.equalsIgnoreCase(jobExecutionInfo.getValidateResultDataStorageType())) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--

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.

-->
<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
<parent>
<artifactId>datavines-engine-flink</artifactId>
<groupId>io.datavines</groupId>
<version>1.0.0-SNAPSHOT</version>
</parent>
<modelVersion>4.0.0</modelVersion>

<artifactId>datavines-engine-flink-api</artifactId>

<dependencies>
<dependency>
<groupId>io.datavines</groupId>
<artifactId>datavines-engine-api</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>io.datavines</groupId>
<artifactId>datavines-common</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge_${scala.binary.version}</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
</dependency>
</dependencies>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,89 @@
/*
* 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 io.datavines.engine.flink.api;

import io.datavines.common.config.CheckResult;
import io.datavines.common.config.Config;
import io.datavines.common.exception.DataVinesException;
import io.datavines.engine.api.env.Execution;
import io.datavines.engine.api.env.RuntimeEnvironment;
import io.datavines.engine.flink.api.stream.FlinkStreamExecution;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;

public class FlinkRuntimeEnvironment implements RuntimeEnvironment {

private StreamExecutionEnvironment env;
private StreamTableEnvironment tableEnv;
private Config config;
private FlinkStreamExecution execution;

public FlinkRuntimeEnvironment() {
this.config = new Config();
this.execution = new FlinkStreamExecution(this);
}

public void setConfig(Config config) {
if (config != null) {
this.config = config;
}
}

public Config getConfig() {
return config;
}

public CheckResult checkConfig() {
return new CheckResult(true, "Configuration check passed");
}

public Execution getExecution() {
return execution;
}

public void prepare() {
try {
env = StreamExecutionEnvironment.getExecutionEnvironment();
tableEnv = StreamTableEnvironment.create(env);
} catch (Exception e) {
throw new DataVinesException("Failed to prepare Flink environment", e);
}
}

public void stop() {
try {
if (env != null) {
// Flink's environment doesn't have a direct cancel method,
// we need to handle job cancellation through JobClient
}
} catch (Exception e) {
throw new DataVinesException("Failed to stop Flink environment", e);
}
}

public String getType() {
return "flink";
}

public StreamExecutionEnvironment getEnv() {
return env;
}

public StreamTableEnvironment getTableEnv() {
return tableEnv;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,101 @@
/*
* 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 io.datavines.engine.flink.api.stream;

import io.datavines.common.config.CheckResult;
import io.datavines.common.config.Config;
import io.datavines.engine.api.component.Component;
import io.datavines.engine.api.env.Execution;
import io.datavines.engine.api.plugin.Plugin;
import io.datavines.engine.flink.api.FlinkRuntimeEnvironment;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.table.api.Table;
import org.apache.flink.table.api.bridge.java.StreamTableEnvironment;
import org.apache.flink.types.Row;

import java.util.List;

public class FlinkStreamExecution implements Execution<FlinkStreamSource, FlinkStreamTransform, FlinkStreamSink>, Plugin {

private final FlinkRuntimeEnvironment flinkEnv;
private Config config;

public FlinkStreamExecution(FlinkRuntimeEnvironment flinkEnv) {
this.flinkEnv = flinkEnv;
this.config = new Config();
}

@Override
public void setConfig(Config config) {
if (config != null) {
this.config = config;
}
}

@Override
public Config getConfig() {
return config;
}

@Override
public CheckResult checkConfig() {
return new CheckResult(true, "Configuration check passed");
}

public String getType() {
return "flink_stream";
}

@Override
public void prepare() throws Exception {
// Initialization if needed
}

@Override
public void execute(List<FlinkStreamSource> sources, List<FlinkStreamTransform> transforms, List<FlinkStreamSink> sinks) throws Exception {
for (FlinkStreamSource source : sources) {
DataStream<Row> sourceStream = source.getData(flinkEnv);
createTemporaryView(source.getClass().getSimpleName(), sourceStream, source.getFieldNames());

DataStream<Row> transformedStream = sourceStream;
for (FlinkStreamTransform transform : transforms) {
transformedStream = transform.process(transformedStream, flinkEnv);
createTemporaryView(transform.getClass().getSimpleName(), transformedStream, transform.getOutputFieldNames());
}

for (FlinkStreamSink sink : sinks) {
sink.output(transformedStream, flinkEnv);
}
}

flinkEnv.getEnv().execute();
}

@Override
public void stop() throws Exception {
// Flink's execution doesn't need explicit stopping
}

private void createTemporaryView(String tableName, DataStream<Row> dataStream, String[] fieldNames) {
StreamTableEnvironment tableEnv = flinkEnv.getTableEnv();
Table table = tableEnv.fromDataStream(dataStream);
for (int i = 0; i < fieldNames.length; i++) {
table = table.as(fieldNames[i]);
}
tableEnv.createTemporaryView(tableName, table);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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 io.datavines.engine.flink.api.stream;

import io.datavines.engine.api.component.Component;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.types.Row;
import io.datavines.engine.flink.api.FlinkRuntimeEnvironment;

public interface FlinkStreamSink extends Component {

/**
* 输出数据流
*/
void output(DataStream<Row> dataStream, FlinkRuntimeEnvironment environment);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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 io.datavines.engine.flink.api.stream;

import io.datavines.engine.api.component.Component;
import org.apache.flink.streaming.api.datastream.DataStream;
import org.apache.flink.types.Row;
import io.datavines.engine.flink.api.FlinkRuntimeEnvironment;

public interface FlinkStreamSource extends Component {

/**
* 获取数据流
*/
DataStream<Row> getData(FlinkRuntimeEnvironment environment);

/**
* 获取数据Schema
*/
String[] getFieldNames();

/**
* 获取数据类型
*/
Class<?>[] getFieldTypes();
}
Loading
Loading