diff --git a/datavines-common/src/main/java/io/datavines/common/ConfigConstants.java b/datavines-common/src/main/java/io/datavines/common/ConfigConstants.java index 4865328e8..0d6b1199c 100644 --- a/datavines-common/src/main/java/io/datavines/common/ConfigConstants.java +++ b/datavines-common/src/main/java/io/datavines/common/ConfigConstants.java @@ -108,7 +108,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"; @@ -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"; } diff --git a/datavines-common/src/main/java/io/datavines/common/config/Config.java b/datavines-common/src/main/java/io/datavines/common/config/Config.java index 2b794429e..feaca2878 100644 --- a/datavines-common/src/main/java/io/datavines/common/config/Config.java +++ b/datavines-common/src/main/java/io/datavines/common/config/Config.java @@ -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 getStringList(String key){ return (List)config.get(key); } @@ -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))); } @@ -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; } diff --git a/datavines-common/src/main/java/io/datavines/common/config/Configurations.java b/datavines-common/src/main/java/io/datavines/common/config/Configurations.java index 44e17fa65..caa74341e 100644 --- a/datavines-common/src/main/java/io/datavines/common/config/Configurations.java +++ b/datavines-common/src/main/java/io/datavines/common/config/Configurations.java @@ -57,4 +57,12 @@ public Float getFloat(String key) { public Float getFloat(String key,String defaultValue) { return Float.valueOf(configuration.getProperty(key,defaultValue)); } + + public long getLong(String key) { + return Long.parseLong(configuration.getProperty(key)); + } + + public long getLong(String key, long defaultValue) { + return Long.parseLong(configuration.getProperty(key, String.valueOf(defaultValue))); + } } diff --git a/datavines-common/src/main/java/io/datavines/common/utils/ProcessUtils.java b/datavines-common/src/main/java/io/datavines/common/utils/ProcessUtils.java index 966f90527..5ee15afc2 100644 --- a/datavines-common/src/main/java/io/datavines/common/utils/ProcessUtils.java +++ b/datavines-common/src/main/java/io/datavines/common/utils/ProcessUtils.java @@ -18,14 +18,18 @@ import java.io.File; import java.io.IOException; +import java.io.InputStreamReader; import java.util.ArrayList; import java.util.List; +import java.util.function.Consumer; import java.util.regex.Matcher; import java.util.regex.Pattern; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.BufferedReader; + /** * mainly used to get the start command line of a process */ @@ -255,4 +259,43 @@ private static boolean needsEscaping(int verificationType, String arg) { } return false; } + + /** + * Print process output to logger + * @param process process + * @param logHandler log handler consumer + * @throws IOException io exception + */ + public static void printProcessOutput(Process process, Consumer> logHandler) throws IOException { + if (process == null || logHandler == null) { + return; + } + + try (BufferedReader inReader = new BufferedReader(new InputStreamReader(process.getInputStream())); + BufferedReader errReader = new BufferedReader(new InputStreamReader(process.getErrorStream()))) { + + List lines = new ArrayList<>(); + String line; + + while ((line = inReader.readLine()) != null) { + lines.add(line); + if (lines.size() >= 100) { // Buffer size to avoid memory issues + logHandler.accept(new ArrayList<>(lines)); + lines.clear(); + } + } + + while ((line = errReader.readLine()) != null) { + lines.add(line); + if (lines.size() >= 100) { + logHandler.accept(new ArrayList<>(lines)); + lines.clear(); + } + } + + if (!lines.isEmpty()) { + logHandler.accept(lines); + } + } + } } diff --git a/datavines-engine/datavines-engine-config/src/main/java/io/datavines/engine/config/BaseJobConfigurationBuilder.java b/datavines-engine/datavines-engine-config/src/main/java/io/datavines/engine/config/BaseJobConfigurationBuilder.java index 38620164c..8b0334428 100644 --- a/datavines-engine/datavines-engine-config/src/main/java/io/datavines/engine/config/BaseJobConfigurationBuilder.java +++ b/datavines-engine/datavines-engine-config/src/main/java/io/datavines/engine/config/BaseJobConfigurationBuilder.java @@ -70,13 +70,13 @@ public void init(Map 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())) { diff --git a/datavines-engine/datavines-engine-executor/src/main/java/io/datavines/engine/executor/core/executor/BaseCommandProcess.java b/datavines-engine/datavines-engine-executor/src/main/java/io/datavines/engine/executor/core/executor/BaseCommandProcess.java index 600e311ea..3301a7643 100644 --- a/datavines-engine/datavines-engine-executor/src/main/java/io/datavines/engine/executor/core/executor/BaseCommandProcess.java +++ b/datavines-engine/datavines-engine-executor/src/main/java/io/datavines/engine/executor/core/executor/BaseCommandProcess.java @@ -43,7 +43,7 @@ public abstract class BaseCommandProcess { /** * process */ - private Process process; + protected Process process; /** * log handler @@ -60,7 +60,7 @@ public abstract class BaseCommandProcess { */ protected Logger logger; - private final Configurations configurations; + protected final Configurations configurations; /** * log list @@ -163,7 +163,7 @@ private void buildProcess(String commandFile) throws IOException{ * @param process process * @return processId */ - private int getProcessId(Process process){ + protected int getProcessId(Process process){ int processId = 0; try { @@ -266,7 +266,7 @@ private void hardKill(int processId) { * get the standard output of the process * @param process process */ - private void parseProcessOutput(Process process) { + protected void parseProcessOutput(Process process) { String threadLoggerInfoName = String.format(LoggerUtils.JOB_LOGGER_THREAD_NAME + "-%s", jobExecutionRequest.getJobExecutionName()); ExecutorService parseProcessOutputExecutorService = ThreadUtils.newDaemonSingleThreadExecutor(threadLoggerInfoName); parseProcessOutputExecutorService.submit(new Runnable(){ @@ -357,4 +357,8 @@ protected List commandOptions() { */ protected abstract void createCommandFileIfNotExists(String execCommand, String commandFile) throws IOException; + protected Consumer> getLogHandler() { + return logHandler; + } + } diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/HEADER b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/HEADER new file mode 100644 index 000000000..8853bce32 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/HEADER @@ -0,0 +1,14 @@ +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. diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/pom.xml new file mode 100644 index 000000000..0864ea39c --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/pom.xml @@ -0,0 +1,59 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-api + + + + io.datavines + datavines-engine-api + ${project.version} + + + + io.datavines + datavines-common + ${project.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink.version} + provided + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/FlinkRuntimeEnvironment.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/FlinkRuntimeEnvironment.java new file mode 100644 index 000000000..83a04a72c --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/FlinkRuntimeEnvironment.java @@ -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; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamExecution.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamExecution.java new file mode 100644 index 000000000..e877d7124 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamExecution.java @@ -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, 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 sources, List transforms, List sinks) throws Exception { + for (FlinkStreamSource source : sources) { + DataStream sourceStream = source.getData(flinkEnv); + createTemporaryView(source.getClass().getSimpleName(), sourceStream, source.getFieldNames()); + + DataStream 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 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); + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSink.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSink.java new file mode 100644 index 000000000..df59f4f0d --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSink.java @@ -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 dataStream, FlinkRuntimeEnvironment environment); +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSource.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSource.java new file mode 100644 index 000000000..11610938d --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamSource.java @@ -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 getData(FlinkRuntimeEnvironment environment); + + /** + * 获取数据Schema + */ + String[] getFieldNames(); + + /** + * 获取数据类型 + */ + Class[] getFieldTypes(); +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamTransform.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamTransform.java new file mode 100644 index 000000000..804ba4a34 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/java/io/datavines/engine/flink/api/stream/FlinkStreamTransform.java @@ -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 FlinkStreamTransform extends Component { + + /** + * 处理数据流 + */ + DataStream process(DataStream dataStream, FlinkRuntimeEnvironment environment); + + /** + * 获取输出Schema + */ + String[] getOutputFieldNames(); + + /** + * 获取输出数据类型 + */ + Class[] getOutputFieldTypes(); +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/resources/META-INF/plugins/io.datavines.engine.api.env.RuntimeEnvironment b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/resources/META-INF/plugins/io.datavines.engine.api.env.RuntimeEnvironment new file mode 100644 index 000000000..298054c2f --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-api/src/main/resources/META-INF/plugins/io.datavines.engine.api.env.RuntimeEnvironment @@ -0,0 +1 @@ +flink=io.datavines.engine.flink.api.FlinkRuntimeEnvironment diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/pom.xml new file mode 100644 index 000000000..77d1aab93 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/pom.xml @@ -0,0 +1,69 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-config + + + + io.datavines + datavines-engine-core + ${project.version} + + + + io.datavines + datavines-engine-api + ${project.version} + + + + io.datavines + datavines-engine-flink-api + ${project.version} + + + + io.datavines + datavines-engine-config + ${project.version} + + + + io.datavines + datavines-connector-api + ${project.version} + + + + org.apache.flink + flink-core + ${flink.version} + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/BaseFlinkConfigurationBuilder.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/BaseFlinkConfigurationBuilder.java new file mode 100644 index 000000000..d2e4d7ff4 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/BaseFlinkConfigurationBuilder.java @@ -0,0 +1,227 @@ +/* + * 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.config; + +import io.datavines.common.config.EnvConfig; +import io.datavines.common.config.SinkConfig; +import io.datavines.common.config.SourceConfig; +import io.datavines.common.config.enums.SinkType; +import io.datavines.common.config.enums.SourceType; +import io.datavines.common.entity.ConnectorParameter; +import io.datavines.common.entity.job.BaseJobParameter; +import io.datavines.common.exception.DataVinesException; +import io.datavines.common.utils.JSONUtils; +import io.datavines.common.utils.StringUtils; +import io.datavines.connector.api.ConnectorFactory; +import io.datavines.engine.common.utils.ParserUtils; +import io.datavines.engine.config.BaseJobConfigurationBuilder; +import io.datavines.spi.PluginLoader; +import lombok.extern.slf4j.Slf4j; +import org.apache.commons.collections4.CollectionUtils; + +import java.util.*; +import java.util.stream.Collectors; + +import static io.datavines.common.CommonConstants.*; +import static io.datavines.common.ConfigConstants.*; +import static io.datavines.common.ConfigConstants.TABLE; + +/** + * + * + * @author dataVines + * @since 2021-07-01 + */ +@Slf4j +public abstract class BaseFlinkConfigurationBuilder extends BaseJobConfigurationBuilder { + + @Override + protected EnvConfig getEnvConfig() { + EnvConfig envConfig = new EnvConfig(); + envConfig.setEngine(jobExecutionInfo.getEngineType()); + Map configMap = envConfig.getConfig(); + if (configMap == null) { + configMap = new HashMap<>(); + } + + ConnectorParameter connectorParameter = jobExecutionParameter.getConnectorParameter(); + String srcConnectorType = ""; + if (connectorParameter != null) { + srcConnectorType = connectorParameter.getType(); + } + + ConnectorParameter connectorParameter2 = jobExecutionParameter.getConnectorParameter2(); + String srcConnectorType2 = ""; + if (connectorParameter2 != null) { + srcConnectorType2 = connectorParameter2.getType(); + } + + envConfig.setConfig(configMap); + return envConfig; + } + + @Override + protected List getSourceConfigs() throws DataVinesException { + List sourceConfigs = new ArrayList<>(); + List metricJobParameterList = jobExecutionParameter.getMetricParameterList(); + boolean isAddValidateResultDataSource = false; + if (CollectionUtils.isNotEmpty(metricJobParameterList)) { + Set sourceConnectorSet = new HashSet<>(); + Set targetConnectorSet = new HashSet<>(); + for (BaseJobParameter parameter : metricJobParameterList) { + String metricUniqueKey = getMetricUniqueKey(parameter); + Map metricInputParameter = metric2InputParameter.get(metricUniqueKey); + if (jobExecutionParameter.getConnectorParameter() != null) { + ConnectorParameter connectorParameter = jobExecutionParameter.getConnectorParameter(); + SourceConfig sourceConfig = new SourceConfig(); + + Map connectorParameterMap = new HashMap<>(connectorParameter.getParameters()); + connectorParameterMap.putAll(metricInputParameter); + + if (connectorParameter.getParameters().get(SCHEMA) != null) { + metricInputParameter.put(SCHEMA, (String)connectorParameter.getParameters().get(SCHEMA)); + } + + metricInputParameter.put(DATABASE_NAME, metricInputParameter.get(DATABASE)); + metricInputParameter.put(TABLE_NAME, metricInputParameter.get(TABLE)); + metricInputParameter.put(COLUMN_NAME, metricInputParameter.get(COLUMN)); + + ConnectorFactory connectorFactory = PluginLoader + .getPluginLoader(ConnectorFactory.class) + .getNewPlugin(connectorParameter.getType()); + + connectorParameterMap.put(TABLE, metricInputParameter.get(TABLE)); + connectorParameterMap.put(DATABASE, metricInputParameter.get(DATABASE)); + connectorParameterMap = connectorFactory.getConnectorParameterConverter().converter(connectorParameterMap); + connectorParameterMap.put(PASSWORD, ParserUtils.encode((String)connectorParameterMap.get(PASSWORD))); + + String outputTable = getOutputTable(metricInputParameter.get(DATABASE), metricInputParameter.get(SCHEMA), metricInputParameter.get(TABLE)); + String tableAlias = getTableAlias(metricInputParameter.get(DATABASE), metricInputParameter.get(SCHEMA), metricInputParameter.get(TABLE), "1"); + connectorParameterMap.put(OUTPUT_TABLE, outputTable); + connectorParameterMap.put(DRIVER, connectorFactory.getDialect().getDriver()); + + metricInputParameter.put(TABLE, outputTable); + metricInputParameter.put(TABLE_ALIAS, tableAlias); + metricInputParameter.put(COLUMN, metricInputParameter.get(COLUMN)); + metricInputParameter.put(REGEX_KEY, "REGEXP(${column}, ${regex})"); + metricInputParameter.put(NOT_REGEX_KEY, "NOT REGEXP(${column}, ${regex})"); + metricInputParameter.put(STRING_TYPE, "STRING"); + metricInputParameter.put(IF_FUNCTION_KEY, "IF"); + metricInputParameter.put(LIMIT_TOP_50_KEY, " LIMIT 50"); + metricInputParameter.put(LENGTH_KEY, "CHARACTER_LENGTH(${column})"); + metricInputParameter.put(SRC_CONNECTOR_TYPE, connectorParameter.getType()); + metricInputParameter.put(ENGINE_TYPE, jobExecutionInfo.getEngineType()); + + String connectorUUID = connectorFactory.getConnectorParameterConverter().getConnectorUUID(connectorParameterMap); + + if (sourceConnectorSet.contains(connectorUUID)) { + continue; + } + + sourceConfig.setPlugin(connectorFactory.getCategory()); + sourceConfig.setConfig(connectorParameterMap); + sourceConfig.setType(SourceType.SOURCE.getDescription()); + sourceConfigs.add(sourceConfig); + sourceConnectorSet.add(connectorUUID); + } + + if (jobExecutionParameter.getConnectorParameter2() != null + && jobExecutionParameter.getConnectorParameter2().getParameters() != null) { + ConnectorParameter connectorParameter2 = jobExecutionParameter.getConnectorParameter2(); + SourceConfig sourceConfig = new SourceConfig(); + + Map connectorParameterMap = new HashMap<>(connectorParameter2.getParameters()); + connectorParameterMap.putAll(metricInputParameter); + + if (connectorParameter2.getParameters().get(SCHEMA) != null) { + metricInputParameter.put(SCHEMA2, (String)connectorParameter2.getParameters().get(SCHEMA)); + } + + ConnectorFactory connectorFactory = PluginLoader + .getPluginLoader(ConnectorFactory.class) + .getNewPlugin(connectorParameter2.getType()); + + connectorParameterMap.put(TABLE, metricInputParameter.get(TABLE2)); + connectorParameterMap.put(DATABASE, metricInputParameter.get(DATABASE2)); + connectorParameterMap = connectorFactory.getConnectorParameterConverter().converter(connectorParameterMap); + connectorParameterMap.put(PASSWORD, ParserUtils.encode((String)connectorParameterMap.get(PASSWORD))); + + String outputTable = getOutputTable(metricInputParameter.get(DATABASE2), + metricInputParameter.get(SCHEMA2), + metricInputParameter.get(TABLE2)) + "_2"; + + String tableAlias = getTableAlias(metricInputParameter.get(DATABASE2), + metricInputParameter.get(SCHEMA2), + metricInputParameter.get(TABLE2), "2"); + + connectorParameterMap.put(OUTPUT_TABLE, outputTable); + connectorParameterMap.put(DRIVER, connectorFactory.getDialect().getDriver()); + + metricInputParameter.put(TABLE2, outputTable); + metricInputParameter.put(TABLE2_ALIAS, tableAlias); + + String connectorUUID = connectorFactory.getConnectorParameterConverter().getConnectorUUID(connectorParameterMap); + + if (targetConnectorSet.contains(connectorUUID)) { + continue; + } + + sourceConfig.setPlugin(connectorFactory.getCategory()); + sourceConfig.setConfig(connectorParameterMap); + sourceConfig.setType(SourceType.SOURCE.getDescription()); + sourceConfigs.add(sourceConfig); + targetConnectorSet.add(connectorUUID); + } + + metric2InputParameter.put(metricUniqueKey, metricInputParameter); + } + } + + return sourceConfigs; + } + + protected String getOutputTable(String database, String schema, String table) { + if (StringUtils.isNotEmpty(schema)) { + return String.format("%s_%s_%s", database, schema, table); + } + return String.format("%s_%s", database, table); + } + + protected String getTableAlias(String database, String schema, String table, String order) { + if (StringUtils.isNotEmpty(schema)) { + return String.format("t%s_%s_%s_%s", order, database, schema, table); + } + return String.format("t%s_%s_%s", order, database, table); + } + + protected SinkConfig getErrorSinkConfig(Map inputParameter) { + if (FILE.equalsIgnoreCase(jobExecutionInfo.getErrorDataStorageType())) { + SinkConfig sinkConfig = new SinkConfig(); + Map configMap = new HashMap<>(); + Map errorDataParameterMap = JSONUtils.toMap(jobExecutionInfo.getErrorDataStorageParameter(),String.class, String.class); + configMap.put(DATA_DIR, errorDataParameterMap.get(DATA_DIR)); + configMap.put(FILE_NAME, inputParameter.get(ERROR_DATA_FILE_NAME)); + configMap.put(COLUMN_SEPARATOR, errorDataParameterMap.get(COLUMN_SEPARATOR)); + configMap.put(LINE_SEPARATOR, errorDataParameterMap.get(LINE_SEPARATOR)); + sinkConfig.setConfig(configMap); + sinkConfig.setType(SinkType.ERROR_DATA.getDescription()); + sinkConfig.setPlugin(FILE); + return sinkConfig; + } + return null; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkConfiguration.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkConfiguration.java new file mode 100644 index 000000000..e2f024a18 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkConfiguration.java @@ -0,0 +1,55 @@ +/* + * 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.config; + +import io.datavines.common.config.BaseConfig; + +public class FlinkConfiguration extends BaseConfig { + + private String jobName; + private String checkpointPath; + private int checkpointInterval = 10000; // default 10s + + @Override + public String getType() { + return "FLINK"; + } + + public String getJobName() { + return jobName; + } + + public void setJobName(String jobName) { + this.jobName = jobName; + } + + public String getCheckpointPath() { + return checkpointPath; + } + + public void setCheckpointPath(String checkpointPath) { + this.checkpointPath = checkpointPath; + } + + public int getCheckpointInterval() { + return checkpointInterval; + } + + public void setCheckpointInterval(int checkpointInterval) { + this.checkpointInterval = checkpointInterval; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkEngineConfig.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkEngineConfig.java new file mode 100644 index 000000000..d4e47eb33 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkEngineConfig.java @@ -0,0 +1,87 @@ +/* + * 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.config; + +import io.datavines.common.config.Config; +import io.datavines.common.config.CheckResult; +import io.datavines.engine.api.plugin.Plugin; +import org.apache.flink.api.common.RuntimeExecutionMode; + +import java.io.Serializable; + +public class FlinkEngineConfig implements Plugin, Serializable { + + private static final long serialVersionUID = 1L; + + private static final String CHECKPOINT_INTERVAL = "flink.checkpoint.interval"; + private static final String PARALLELISM = "flink.parallelism"; + private static final String RESTART_ATTEMPTS = "flink.restart.attempts"; + private static final String RESTART_DELAY = "flink.restart.delay"; + private static final String STATE_BACKEND = "flink.state.backend"; + private static final String CHECKPOINT_PATH = "flink.checkpoint.path"; + private static final String EXECUTION_MODE = "flink.execution.mode"; + + private Config config; + + public FlinkEngineConfig() { + this.config = new Config(); + } + + @Override + public void setConfig(Config config) { + this.config = config != null ? config : new Config(); + } + + @Override + public Config getConfig() { + return config; + } + + @Override + public CheckResult checkConfig() { + return new CheckResult(true, ""); + } + + public long getCheckpointInterval() { + return config.getLong(CHECKPOINT_INTERVAL, 10000L); + } + + public int getParallelism() { + return config.getInt(PARALLELISM, 1); + } + + public int getRestartAttempts() { + return config.getInt(RESTART_ATTEMPTS, 3); + } + + public long getRestartDelay() { + return config.getLong(RESTART_DELAY, 10000L); + } + + public String getStateBackend() { + return config.getString(STATE_BACKEND, "memory"); + } + + public String getCheckpointPath() { + return config.getString(CHECKPOINT_PATH, ""); + } + + public RuntimeExecutionMode getExecutionMode() { + String mode = config.getString(EXECUTION_MODE, "STREAMING"); + return RuntimeExecutionMode.valueOf(mode.toUpperCase()); + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSingleTableConfigurationBuilder.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSingleTableConfigurationBuilder.java new file mode 100644 index 000000000..c839d1a9e --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSingleTableConfigurationBuilder.java @@ -0,0 +1,140 @@ +/* + * 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.config; + +import io.datavines.common.config.EnvConfig; +import io.datavines.common.config.SinkConfig; +import io.datavines.common.config.SourceConfig; +import io.datavines.common.config.enums.SinkType; +import io.datavines.common.entity.job.BaseJobParameter; +import io.datavines.common.exception.DataVinesException; +import io.datavines.common.utils.StringUtils; +import io.datavines.engine.config.MetricParserUtils; +import io.datavines.metric.api.ExpectedValue; +import io.datavines.spi.PluginLoader; +import org.apache.commons.collections4.CollectionUtils; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import static io.datavines.common.ConfigConstants.*; + +public class FlinkSingleTableConfigurationBuilder extends BaseFlinkConfigurationBuilder { + + + @Override + public void buildEnvConfig() { + EnvConfig envConfig = new EnvConfig(); + envConfig.setEngine("flink"); + configuration.setEnvConfig(envConfig); + } + + @Override + public void buildSinkConfigs() throws DataVinesException { + List sinkConfigs = new ArrayList<>(); + + List metricJobParameterList = jobExecutionParameter.getMetricParameterList(); + if (CollectionUtils.isNotEmpty(metricJobParameterList)) { + for (BaseJobParameter parameter : metricJobParameterList) { + String metricUniqueKey = getMetricUniqueKey(parameter); + Map metricInputParameter = metric2InputParameter.get(metricUniqueKey); + if (metricInputParameter == null) { + continue; + } + + // 确保必要的参数存在 + if (!metricInputParameter.containsKey(METRIC_NAME) && parameter.getMetricType() != null) { + metricInputParameter.put(METRIC_NAME, parameter.getMetricType()); + } + + metricInputParameter.put(METRIC_UNIQUE_KEY, metricUniqueKey); + String expectedType = "local_" + parameter.getExpectedType(); + ExpectedValue expectedValue = PluginLoader + .getPluginLoader(ExpectedValue.class) + .getNewPlugin(expectedType); + + // 只有在确保必要参数存在的情况下才生成 uniqueCode + if (metricInputParameter.containsKey(METRIC_NAME)) { + metricInputParameter.put(UNIQUE_CODE, StringUtils.wrapperSingleQuotes(MetricParserUtils.generateUniqueCode(metricInputParameter))); + } + + // Get the actual value storage parameter + String actualValueSinkSql = FlinkSinkSqlBuilder.getActualValueSql() + .replace("${actual_value}", "${actual_value_" + metricUniqueKey + "}"); + SinkConfig actualValueSinkConfig = getValidateResultDataSinkConfig( + expectedValue, actualValueSinkSql, "dv_actual_values", metricInputParameter); + + if (actualValueSinkConfig != null) { + actualValueSinkConfig.setType(SinkType.ACTUAL_VALUE.getDescription()); + sinkConfigs.add(actualValueSinkConfig); + } + + String taskSinkSql = FlinkSinkSqlBuilder.getDefaultSinkSql() + .replace("${actual_value}", "${actual_value_" + metricUniqueKey + "}") + .replace("${expected_value}", "${expected_value_" + metricUniqueKey + "}"); + + // Get the task data storage parameter + SinkConfig taskResultSinkConfig = getValidateResultDataSinkConfig( + expectedValue, taskSinkSql, "dv_job_execution_result", metricInputParameter); + if (taskResultSinkConfig != null) { + taskResultSinkConfig.setType(SinkType.VALIDATE_RESULT.getDescription()); + // 设置默认状态为未知(NONE) + taskResultSinkConfig.getConfig().put("default_state", "0"); + // 添加其他必要参数 + taskResultSinkConfig.getConfig().put("metric_type", "single_table"); + taskResultSinkConfig.getConfig().put("metric_name", metricInputParameter.get(METRIC_NAME)); + taskResultSinkConfig.getConfig().put("metric_dimension", metricInputParameter.get(METRIC_DIMENSION)); + taskResultSinkConfig.getConfig().put("database_name", metricInputParameter.get(DATABASE)); + taskResultSinkConfig.getConfig().put("table_name", metricInputParameter.get(TABLE)); + taskResultSinkConfig.getConfig().put("column_name", metricInputParameter.get(COLUMN)); + taskResultSinkConfig.getConfig().put("expected_type", metricInputParameter.get(EXPECTED_TYPE)); + taskResultSinkConfig.getConfig().put("result_formula", metricInputParameter.get(RESULT_FORMULA)); + sinkConfigs.add(taskResultSinkConfig); + } + + // Get the error data storage parameter if needed + if (StringUtils.isNotEmpty(jobExecutionInfo.getErrorDataStorageType()) + && StringUtils.isNotEmpty(jobExecutionInfo.getErrorDataStorageParameter())) { + SinkConfig errorDataSinkConfig = getErrorSinkConfig(metricInputParameter); + if (errorDataSinkConfig != null) { + errorDataSinkConfig.setType(SinkType.ERROR_DATA.getDescription()); + sinkConfigs.add(errorDataSinkConfig); + } + } + } + } + + configuration.setSinkParameters(sinkConfigs); + } + + @Override + public void buildTransformConfigs() { + // No transform configs needed for single table configuration + } + + @Override + public void buildSourceConfigs() throws DataVinesException { + List sourceConfigs = getSourceConfigs(); + configuration.setSourceParameters(sourceConfigs); + } + + @Override + public void buildName() { + // Use default name from base implementation + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSinkSqlBuilder.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSinkSqlBuilder.java new file mode 100644 index 000000000..568dad4b6 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/java/io/datavines/engine/flink/config/FlinkSinkSqlBuilder.java @@ -0,0 +1,62 @@ +/* + * 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.config; + +public class FlinkSinkSqlBuilder { + + private FlinkSinkSqlBuilder() { + throw new IllegalStateException("Utility class"); + } + + public static String getActualValueSql() { + return "select\n" + + " '${job_execution_id}' as job_execution_id,\n" + + " '${metric_unique_key}' as metric_unique_key,\n" + + " '${unique_code}' as unique_code,\n" + + " ${actual_value} as actual_value,\n" + + " cast(null as string) as expected_value,\n" + + " cast(null as string) as operator,\n" + + " cast(null as string) as threshold,\n" + + " cast(null as string) as check_type,\n" + + " CURRENT_TIMESTAMP as create_time,\n" + + " CURRENT_TIMESTAMP as update_time\n" + + "from ${table_name}"; + } + + public static String getDefaultSinkSql() { + return "select\n" + + " '${job_execution_id}' as job_execution_id,\n" + + " '${metric_unique_key}' as metric_unique_key,\n" + + " '${unique_code}' as unique_code,\n" + + " CASE WHEN ${actual_value} IS NULL THEN NULL ELSE ${actual_value} END as actual_value,\n" + + " CASE WHEN ${expected_value} IS NULL THEN NULL ELSE ${expected_value} END as expected_value,\n" + + " '${metric_type}' as metric_type,\n" + + " '${metric_name}' as metric_name,\n" + + " '${metric_dimension}' as metric_dimension,\n" + + " '${database_name}' as database_name,\n" + + " '${table_name}' as table_name,\n" + + " '${column_name}' as column_name,\n" + + " '${operator}' as operator,\n" + + " '${threshold}' as threshold,\n" + + " '${expected_type}' as expected_type,\n" + + " '${result_formula}' as result_formula,\n" + + " CASE WHEN ${actual_value} IS NULL THEN '${default_state}' ELSE NULL END as state,\n" + + " CURRENT_TIMESTAMP as create_time,\n" + + " CURRENT_TIMESTAMP as update_time\n" + + "from ${table_name} full join ${expected_table}"; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor new file mode 100644 index 000000000..68045cb12 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor @@ -0,0 +1 @@ +flink=io.datavines.engine.flink.FlinkEngineExecutor diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/services/io.datavines.engine.config.JobConfigurationBuilder b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/services/io.datavines.engine.config.JobConfigurationBuilder new file mode 100644 index 000000000..394a4be0f --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-config/src/main/resources/META-INF/services/io.datavines.engine.config.JobConfigurationBuilder @@ -0,0 +1 @@ +flink_single_table=io.datavines.engine.flink.config.FlinkSingleTableConfigurationBuilder diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/pom.xml new file mode 100644 index 000000000..80c6ecb2f --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/pom.xml @@ -0,0 +1,84 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-core + + + + io.datavines + datavines-engine-flink-api + ${project.version} + + + io.datavines + datavines-engine-api + ${project.version} + + + io.datavines + datavines-engine-core + ${project.version} + + + org.apache.flink + flink-connector-jdbc_${scala.binary.version} + ${flink.version} + + + io.datavines + datavines-engine-common + ${project.version} + + + io.datavines + datavines-connector-api + ${project.version} + + + io.datavines + datavines-engine-config + ${project.version} + + + io.datavines + datavines-common + ${project.version} + + + io.datavines + datavines-engine-flink-executor + ${project.version} + + + io.datavines + datavines-engine-flink-config + ${project.version} + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/FlinkEngine.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/FlinkEngine.java new file mode 100644 index 000000000..fbbc1727a --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/FlinkEngine.java @@ -0,0 +1,86 @@ +/* + * 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; + +import io.datavines.common.config.Config; +import io.datavines.common.config.Configurations; +import io.datavines.common.entity.JobExecutionRequest; +import io.datavines.common.entity.ProcessResult; + +import org.slf4j.Logger; + +import io.datavines.common.config.CheckResult; +import io.datavines.engine.api.engine.EngineExecutor; +import io.datavines.engine.flink.config.FlinkEngineConfig; +import io.datavines.engine.flink.executor.FlinkEngineExecutor; + +public class FlinkEngine implements EngineExecutor { + + private FlinkEngineConfig flinkEngineConfig; + private FlinkEngineExecutor executor; + + public FlinkEngine() { + this.flinkEngineConfig = new FlinkEngineConfig(); + this.executor = new FlinkEngineExecutor(); + } + + public void init(JobExecutionRequest jobExecutionRequest, Logger logger, Configurations configurations) throws Exception { + executor.init(jobExecutionRequest, logger, configurations); + } + + public void execute() throws Exception { + executor.execute(); + } + + public void after() throws Exception { + executor.after(); + } + + public void cancel() throws Exception { + executor.cancel(); + } + + public boolean isCancel() throws Exception { + return executor.isCancel(); + } + + public ProcessResult getProcessResult() { + return executor.getProcessResult(); + } + + public JobExecutionRequest getTaskRequest() { + return executor.getTaskRequest(); + } + + public String getName() { + return "flink"; + } + + public void setConfig(Config config) { + if (config != null) { + this.flinkEngineConfig.setConfig(config); + } + } + + public Config getConfig() { + return this.flinkEngineConfig.getConfig(); + } + + public CheckResult checkConfig() { + return this.flinkEngineConfig.checkConfig(); + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/core/FlinkDataVinesBootstrap.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/core/FlinkDataVinesBootstrap.java new file mode 100644 index 000000000..5a0764385 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/java/io/datavines/engine/flink/core/FlinkDataVinesBootstrap.java @@ -0,0 +1,33 @@ +/* + * 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.core; + +import java.util.Base64; + +import io.datavines.engine.core.BaseDataVinesBootstrap; + +public class FlinkDataVinesBootstrap extends BaseDataVinesBootstrap { + + public static void main(String[] args) { + FlinkDataVinesBootstrap bootstrap = new FlinkDataVinesBootstrap(); + if (args.length == 1) { + String arg = args[0]; + args[0] = new String(Base64.getDecoder().decode(arg)); + bootstrap.execute(args); + } + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/resources/META-INF/services/io.datavines.engine.api.Engine b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/resources/META-INF/services/io.datavines.engine.api.Engine new file mode 100644 index 000000000..0efce9379 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-core/src/main/resources/META-INF/services/io.datavines.engine.api.Engine @@ -0,0 +1 @@ +flink=io.datavines.engine.flink.FlinkEngine diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/pom.xml new file mode 100644 index 000000000..260054fe5 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/pom.xml @@ -0,0 +1,59 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-executor + + + + io.datavines + datavines-engine-api + ${project.version} + + + io.datavines + datavines-engine-executor + ${project.version} + + + io.datavines + datavines-engine-core + ${project.version} + + + io.datavines + datavines-engine-flink-api + ${project.version} + + + io.datavines + datavines-common + ${project.version} + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkCommandProcess.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkCommandProcess.java new file mode 100644 index 000000000..0e7b73272 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkCommandProcess.java @@ -0,0 +1,200 @@ +/* + * 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.executor; + +import io.datavines.common.config.Configurations; +import io.datavines.common.entity.JobExecutionRequest; +import io.datavines.common.entity.ProcessResult; +import io.datavines.common.enums.ExecutionStatus; +import io.datavines.common.utils.OSUtils; +import io.datavines.common.utils.ProcessUtils; +import io.datavines.common.utils.YarnUtils; +import io.datavines.engine.executor.core.executor.BaseCommandProcess; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.function.Consumer; + +public class FlinkCommandProcess extends BaseCommandProcess { + + private static final String FLINK_COMMAND = "flink"; + private long timeout; + + public FlinkCommandProcess(Consumer> logHandler, + Logger logger, + JobExecutionRequest jobExecutionRequest, + Configurations configurations) { + super(logHandler, logger, jobExecutionRequest, configurations); + } + + public void setTimeout(long timeout) { + this.timeout = timeout; + } + + private ProcessBuilder buildFlinkProcessBuilder(String command) throws IOException { + List commandList = new ArrayList<>(); + + // Get flink home from configurations or environment variable + String flinkHome = configurations.getString("flink.home", System.getenv("FLINK_HOME")); + if (StringUtils.isEmpty(flinkHome)) { + throw new IOException("FLINK_HOME is not set in either configurations or environment variables"); + } + + // Build the flink command path + String flinkCmd = Paths.get(flinkHome, "bin", OSUtils.isWindows() ? "flink.cmd" : "flink").toString(); + + // Split the command while preserving quoted spaces + String[] cmdArray = command.split("\\s+(?=(?:[^\"]*\"[^\"]*\")*[^\"]*$)"); + commandList.add(flinkCmd); + for (String cmd : cmdArray) { + // Remove quotes + cmd = cmd.replaceAll("^\"|\"$", ""); + if (!cmd.trim().isEmpty()) { + commandList.add(cmd); + } + } + + ProcessBuilder processBuilder = new ProcessBuilder(commandList); + processBuilder.directory(new File(jobExecutionRequest.getExecuteFilePath())); + + // Set environment variables + Map env = processBuilder.environment(); + env.put("FLINK_HOME", flinkHome); + + // Add to PATH + String path = env.get("PATH"); + if (path != null) { + path = Paths.get(flinkHome, "bin") + File.pathSeparator + path; + env.put("PATH", path); + } + + return processBuilder; + } + + @Override + public ProcessResult run(String executeCommand) { + if (StringUtils.isEmpty(executeCommand)) { + ProcessResult result = new ProcessResult(); + result.setExitStatusCode(ExecutionStatus.FAILURE.getCode()); + return result; + } + + try { + ProcessBuilder processBuilder = buildFlinkProcessBuilder(executeCommand); + processBuilder.redirectErrorStream(true); + + // 打印命令用于调试 + logger.info("Executing command: {}", String.join(" ", processBuilder.command())); + + Process process = processBuilder.start(); + ProcessResult result = new ProcessResult(); + + try { + // 启动日志处理线程 + startLogHandler(process); + + if (timeout > 0) { + boolean completed = process.waitFor(timeout, TimeUnit.MILLISECONDS); + if (!completed) { + process.destroyForcibly(); + throw new IOException("Process timed out after " + timeout + "ms"); + } + } else { + process.waitFor(); + } + + int exitCode = process.exitValue(); + result.setExitStatusCode(exitCode); + + // 获取Yarn应用ID(如果有) + String appId = YarnUtils.getYarnAppId(jobExecutionRequest.getTenantCode(), + jobExecutionRequest.getJobExecutionUniqueId()); + if (StringUtils.isNotEmpty(appId)) { + result.setApplicationId(appId); + // 根据Yarn状态确定最终状态 + if (exitCode == 0) { + result.setExitStatusCode(YarnUtils.isSuccessOfYarnState(appId) ? + ExecutionStatus.SUCCESS.getCode() : ExecutionStatus.FAILURE.getCode()); + } + } + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Process was interrupted", e); + } + + return result; + } catch (IOException e) { + throw new RuntimeException("Error executing process", e); + } + } + + private void startLogHandler(Process process) { + if (getLogHandler() != null) { + Thread logThread = new Thread(() -> { + try { + ProcessUtils.printProcessOutput(process, getLogHandler()); + } catch (IOException e) { + logger.error("Error handling process output", e); + } + }); + logThread.setDaemon(true); + logThread.start(); + } + } + + @Override + protected String commandInterpreter() { + // TODO Auto-generated method stub + throw new UnsupportedOperationException("Unimplemented method 'commandInterpreter'"); + } + + @Override + protected String buildCommandFilePath() { + // TODO Auto-generated method stub + throw new UnsupportedOperationException("Unimplemented method 'buildCommandFilePath'"); + } + + @Override + protected void createCommandFileIfNotExists(String execCommand, String commandFile) throws IOException { + // TODO Auto-generated method stub + throw new UnsupportedOperationException("Unimplemented method 'createCommandFileIfNotExists'"); + } + + /** + * Cleanup any temporary files created during the Flink job execution + */ + public void cleanupTempFiles() { + String commandFile = buildCommandFilePath(); + if (commandFile != null) { + File file = new File(commandFile); + if (file.exists() && file.isFile()) { + if (!file.delete()) { + logger.warn("Failed to delete temporary command file: {}", commandFile); + } + } + } + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkEngineExecutor.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkEngineExecutor.java new file mode 100644 index 000000000..ec986471c --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/FlinkEngineExecutor.java @@ -0,0 +1,283 @@ +/* + * 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.executor; + +import org.slf4j.Logger; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import io.datavines.common.config.Configurations; +import io.datavines.common.entity.JobExecutionRequest; +import io.datavines.common.entity.ProcessResult; +import io.datavines.common.utils.JSONUtils; +import io.datavines.common.utils.LoggerUtils; +import io.datavines.common.utils.OSUtils; +import io.datavines.engine.executor.core.base.AbstractYarnEngineExecutor; +import io.datavines.engine.executor.core.executor.BaseCommandProcess; +import io.datavines.common.utils.YarnUtils; +import org.apache.commons.lang3.StringUtils; + +import java.io.File; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; + +public class FlinkEngineExecutor extends AbstractYarnEngineExecutor { + + private static final String FLINK_COMMAND = "flink"; + private Configurations configurations; + private JobExecutionRequest jobExecutionRequest; + private Logger logger; + private ProcessResult processResult; + private BaseCommandProcess shellCommandProcess; + private boolean cancel; + + @Override + public void init(JobExecutionRequest jobExecutionRequest, Logger logger, Configurations configurations) throws Exception { + String threadLoggerInfoName = String.format(LoggerUtils.JOB_LOG_INFO_FORMAT, jobExecutionRequest.getJobExecutionUniqueId()); + Thread.currentThread().setName(threadLoggerInfoName); + + this.jobExecutionRequest = jobExecutionRequest; + this.logger = logger; + this.configurations = configurations; + this.processResult = new ProcessResult(); + this.shellCommandProcess = new FlinkCommandProcess( + this::logHandle, + logger, + jobExecutionRequest, + configurations + ); + } + + @Override + public void execute() throws Exception { + try { + String command = buildCommand(); + logger.info("flink task command: {}", command); + this.processResult = shellCommandProcess.run(command); + logger.info("process result: {}", JSONUtils.toJsonString(this.processResult)); + } catch (Exception e) { + logger.error("yarn process failure", e); + throw e; + } + } + + @Override + protected String buildCommand() { + List commandParts = new ArrayList<>(); + + // Get FLINK_HOME from configurations or environment variable + String flinkHome = configurations.getString("flink.home", System.getenv("FLINK_HOME")); + if (StringUtils.isEmpty(flinkHome)) { + // Try to find flink in common locations + String[] commonPaths = { + "C:\\Program Files\\flink", + "C:\\flink", + "/opt/flink", + "/usr/local/flink" + }; + + for (String path : commonPaths) { + if (new File(path).exists()) { + flinkHome = path; + break; + } + } + + if (StringUtils.isEmpty(flinkHome)) { + throw new RuntimeException("FLINK_HOME is not set and Flink installation could not be found in common locations"); + } + logger.info("FLINK_HOME not set, using detected path: {}", flinkHome); + } + + // Build the flink command + String flinkCmd = Paths.get(flinkHome, "bin", OSUtils.isWindows() ? "flink.cmd" : "flink").toString(); + if (!new File(flinkCmd).exists()) { + throw new RuntimeException("Flink command not found at: " + flinkCmd); + } + + // Parse application parameters + String deployMode = "local"; // Default to local mode + JsonNode envNode = null; + try { + String applicationParameter = jobExecutionRequest.getApplicationParameter(); + if (StringUtils.isNotEmpty(applicationParameter)) { + ObjectMapper mapper = new ObjectMapper(); + JsonNode jsonNode = mapper.readTree(applicationParameter); + envNode = jsonNode.get("env"); + if (envNode != null && envNode.has("deployMode")) { + deployMode = envNode.get("deployMode").asText(); + } + } + } catch (Exception e) { + logger.warn("Failed to parse applicationParameter, using default mode: local", e); + } + + // If applicationParameter doesn't specify deployMode, get from configurations + if ("local".equals(deployMode)) { + deployMode = configurations.getString("deployMode", "local"); + } + + logger.info("Using deploy mode: {}", deployMode); + + // Build command parts + commandParts.add(flinkCmd); + + // Add run command and deployment mode specific args + switch (deployMode.toLowerCase()) { + case "yarn-session": + commandParts.add("run"); + commandParts.add("-m"); + commandParts.add("yarn-session"); + addYarnConfig(commandParts, envNode); + break; + case "yarn-per-job": + commandParts.add("run"); + commandParts.add("-m"); + commandParts.add("yarn-per-job"); + addYarnConfig(commandParts, envNode); + break; + case "yarn-application": + commandParts.add("run-application"); + commandParts.add("-t"); + commandParts.add("yarn-application"); + addYarnConfig(commandParts, envNode); + break; + default: + // Local mode + commandParts.add("run"); + break; + } + + // Add basic parameters + int parallelism = configurations.getInt("parallelism", 1); + commandParts.add("-p"); + commandParts.add(String.valueOf(parallelism)); + + // Add main class + String mainClass = configurations.getString("mainClass", "io.datavines.engine.flink.core.FlinkDataVinesBootstrap"); + commandParts.add("-c"); + commandParts.add(mainClass); + + // Add main jar + String mainJar = configurations.getString("mainJar"); + if (StringUtils.isEmpty(mainJar)) { + mainJar = Paths.get(flinkHome, "lib", "datavines-flink-core.jar").toString(); + if (!new File(mainJar).exists()) { + throw new RuntimeException("Main jar not found at: " + mainJar); + } + } + commandParts.add(mainJar); + + // Add program arguments if any + String programArgs = configurations.getString("programArgs"); + if (StringUtils.isNotEmpty(programArgs)) { + commandParts.add(programArgs); + } + + return String.join(" ", commandParts); + } + + private void addYarnConfig(List commandParts, JsonNode envNode) { + commandParts.add("-Dyarn.application.name=" + jobExecutionRequest.getJobExecutionName()); + + // Add memory configuration + String jobManagerMemory = "1024m"; + String taskManagerMemory = "1024m"; + + if (envNode != null) { + if (envNode.has("jobmanager.memory.process.size")) { + jobManagerMemory = envNode.get("jobmanager.memory.process.size").asText("1024m"); + } + if (envNode.has("taskmanager.memory.process.size")) { + taskManagerMemory = envNode.get("taskmanager.memory.process.size").asText("1024m"); + } + } + + commandParts.add("-Djobmanager.memory.process.size=" + jobManagerMemory); + commandParts.add("-Dtaskmanager.memory.process.size=" + taskManagerMemory); + } + + @Override + public void after() { + try { + if (shellCommandProcess != null) { + ((FlinkCommandProcess)shellCommandProcess).cleanupTempFiles(); + } + } catch (Exception e) { + logger.error("Error in after execution", e); + } + } + + @Override + public ProcessResult getProcessResult() { + return this.processResult; + } + + @Override + public JobExecutionRequest getTaskRequest() { + return this.jobExecutionRequest; + } + + public String getApplicationId() { + return processResult != null ? processResult.getApplicationId() : null; + } + + public String getApplicationUrl() { + return processResult != null ? YarnUtils.getApplicationUrl(processResult.getApplicationId()) : null; + } + + @Override + public void cancel() throws Exception { + cancel = true; + if (shellCommandProcess != null) { + shellCommandProcess.cancel(); + } + killYarnApplication(); + } + + private void killYarnApplication() { + try { + String applicationId = YarnUtils.getYarnAppId(jobExecutionRequest.getTenantCode(), + jobExecutionRequest.getJobExecutionUniqueId()); + + if (StringUtils.isNotEmpty(applicationId)) { + String cmd = String.format("sudo -u %s yarn application -kill %s", + jobExecutionRequest.getTenantCode(), + applicationId); + + logger.info("Killing yarn application: {}", applicationId); + Runtime.getRuntime().exec(cmd); + } + } catch (Exception e) { + logger.error("Failed to kill yarn application", e); + } + } + + @Override + public void logHandle(List logs) { + if (logs != null && !logs.isEmpty()) { + for (String log : logs) { + logger.info(log); + } + } + } + + @Override + public boolean isCancel() { + return this.cancel; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkArgsUtils.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkArgsUtils.java new file mode 100644 index 000000000..e0b33ef4e --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkArgsUtils.java @@ -0,0 +1,96 @@ +/* + * 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.executor.parameter; + +import java.util.ArrayList; +import java.util.List; + +public class FlinkArgsUtils { + + private static final String FLINK_LOCAL = "local"; + private static final String FLINK_YARN_SESSION = "yarn-session"; + private static final String FLINK_YARN_PER_JOB = "yarn-per-job"; + private static final String FLINK_YARN_APPLICATION = "yarn-application"; + + private FlinkArgsUtils() { + throw new IllegalStateException("Utility class"); + } + + public static List buildArgs(FlinkParameters param) { + List args = new ArrayList<>(); + + // Add run command based on deployment mode + String deployMode = param.getDeployMode(); + if (deployMode == null || deployMode.isEmpty()) { + deployMode = FLINK_LOCAL; // Default to local mode + } + + switch (deployMode.toLowerCase()) { + case "yarn-session": + args.add("run"); + args.add("-m"); + args.add("yarn-session"); + break; + case "yarn-per-job": + args.add("run"); + args.add("-m"); + args.add("yarn-per-job"); + break; + case "yarn-application": + args.add("run-application"); + args.add("-t"); + args.add("yarn-application"); + break; + case "local": + default: + args.add("run"); + break; + } + + // Add parallelism + if (param.getParallelism() > 0) { + args.add("-p"); + args.add(String.valueOf(param.getParallelism())); + } + + // Add job name if specified (only for YARN modes) + if (!FLINK_LOCAL.equals(deployMode) && param.getJobName() != null && !param.getJobName().isEmpty()) { + args.add("-Dyarn.application.name=" + param.getJobName()); + } + + // Add yarn queue if specified (only for YARN modes) + if (!FLINK_LOCAL.equals(deployMode) && param.getYarnQueue() != null && !param.getYarnQueue().isEmpty()) { + args.add("-Dyarn.application.queue=" + param.getYarnQueue()); + } + + // Add main class + if (param.getMainClass() != null && !param.getMainClass().isEmpty()) { + args.add("-c"); + args.add(param.getMainClass()); + } + + // Add jar file + args.add(param.getMainJar()); + + // Add program arguments if any + if (param.getMainArgs() != null && !param.getMainArgs().isEmpty()) { + args.add(param.getMainArgs()); + } + + return args; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkParameters.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkParameters.java new file mode 100644 index 000000000..ea84c4dd2 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/java/io/datavines/engine/flink/executor/parameter/FlinkParameters.java @@ -0,0 +1,84 @@ +/* + * 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.executor.parameter; + +public class FlinkParameters { + + private String mainJar; + private String mainClass; + private String deployMode; + private String mainArgs; + private String yarnQueue; + private String jobName; + private int parallelism = 1; + + public String getMainJar() { + return mainJar; + } + + public void setMainJar(String mainJar) { + this.mainJar = mainJar; + } + + public String getMainClass() { + return mainClass; + } + + public void setMainClass(String mainClass) { + this.mainClass = mainClass; + } + + public String getDeployMode() { + return deployMode; + } + + public void setDeployMode(String deployMode) { + this.deployMode = deployMode; + } + + public String getMainArgs() { + return mainArgs; + } + + public void setMainArgs(String mainArgs) { + this.mainArgs = mainArgs; + } + + public String getYarnQueue() { + return yarnQueue; + } + + public void setYarnQueue(String yarnQueue) { + this.yarnQueue = yarnQueue; + } + + public String getJobName() { + return jobName; + } + + public void setJobName(String jobName) { + this.jobName = jobName; + } + + public int getParallelism() { + return parallelism; + } + + public void setParallelism(int parallelism) { + this.parallelism = parallelism; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor new file mode 100644 index 000000000..931d94a8b --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-executor/src/main/resources/META-INF/plugins/io.datavines.engine.api.engine.EngineExecutor @@ -0,0 +1 @@ +flink=io.datavines.engine.flink.executor.FlinkEngineExecutor diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/pom.xml new file mode 100644 index 000000000..87507b3c3 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/pom.xml @@ -0,0 +1,58 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-jdbc + + + + io.datavines + datavines-engine-flink-core + ${project.version} + + + + + org.apache.flink + flink-connector-jdbc_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + + + + mysql + mysql-connector-java + 8.0.16 + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/sink/JdbcSink.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/sink/JdbcSink.java new file mode 100644 index 000000000..091756343 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/sink/JdbcSink.java @@ -0,0 +1,90 @@ +/* + * 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.jdbc.sink; + +import org.apache.flink.connector.jdbc.JdbcConnectionOptions; +import org.apache.flink.connector.jdbc.JdbcExecutionOptions; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.types.Row; + +import io.datavines.common.config.Config; +import io.datavines.common.config.CheckResult; +import io.datavines.engine.api.env.RuntimeEnvironment; +import io.datavines.engine.flink.api.FlinkRuntimeEnvironment; +import io.datavines.engine.flink.api.stream.FlinkStreamSink; + +public class JdbcSink implements FlinkStreamSink { + + private String driverName; + private String jdbcUrl; + private String username; + private String password; + private String query; + private Config config = new Config(); + + @Override + public void output(DataStream dataStream, FlinkRuntimeEnvironment environment) { + dataStream.addSink(org.apache.flink.connector.jdbc.JdbcSink.sink( + query, + (statement, row) -> { + // Need to be implemented based on actual schema + for (int i = 0; i < row.getArity(); i++) { + statement.setObject(i + 1, row.getField(i)); + } + }, + JdbcExecutionOptions.builder() + .withBatchSize(1000) + .withBatchIntervalMs(200) + .withMaxRetries(5) + .build(), + new JdbcConnectionOptions.JdbcConnectionOptionsBuilder() + .withUrl(jdbcUrl) + .withDriverName(driverName) + .withUsername(username) + .withPassword(password) + .build() + )); + } + + @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, ""); + } + + @Override + public void prepare(RuntimeEnvironment env) throws Exception { + if (config != null) { + this.driverName = config.getString("driver"); + this.jdbcUrl = config.getString("url"); + this.username = config.getString("username"); + this.password = config.getString("password"); + this.query = config.getString("query"); + } + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/source/JdbcSource.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/source/JdbcSource.java new file mode 100644 index 000000000..1c00e69da --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-jdbc/src/main/java/io/datavines/engine/flink/jdbc/source/JdbcSource.java @@ -0,0 +1,122 @@ +/* + * 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.jdbc.source; + +import org.apache.flink.connector.jdbc.JdbcInputFormat; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.types.Row; + +import java.util.*; +import java.util.stream.Collectors; + +import io.datavines.common.config.Config; +import io.datavines.common.config.CheckResult; +import io.datavines.common.utils.StringUtils; +import io.datavines.common.utils.TypesafeConfigUtils; +import io.datavines.engine.api.env.RuntimeEnvironment; +import io.datavines.engine.flink.api.FlinkRuntimeEnvironment; +import io.datavines.engine.flink.api.stream.FlinkStreamSource; +import static io.datavines.common.ConfigConstants.*; + +public class JdbcSource implements FlinkStreamSource { + + private Config config = new Config(); + + private JdbcInputFormat buildJdbcInputFormat() { + Properties properties = new Properties(); + properties.setProperty(USER, config.getString(USER)); + properties.setProperty(DRIVER, config.getString(DRIVER)); + String password = config.getString(PASSWORD); + if (!StringUtils.isEmptyOrNullStr(password)) { + properties.put(PASSWORD, password); + } + + Config jdbcConfig = TypesafeConfigUtils.extractSubConfigThrowable(config, "jdbc.", false); + if (!jdbcConfig.isEmpty()) { + jdbcConfig.entrySet().forEach(x -> { + properties.put(x.getKey(), String.valueOf(x.getValue())); + }); + } + + return JdbcInputFormat.buildJdbcInputFormat() + .setDrivername(properties.getProperty(DRIVER)) + .setDBUrl(config.getString(URL)) + .setUsername(properties.getProperty(USER)) + .setPassword(properties.getProperty(PASSWORD)) + .setQuery(config.getString(TABLE)) + .setRowTypeInfo(null) // Need to be implemented based on actual schema + .finish(); + } + + @Override + public DataStream getData(FlinkRuntimeEnvironment environment) { + return environment.getEnv() + .createInput(buildJdbcInputFormat()); + } + + @Override + public String[] getFieldNames() { + // TODO: Implement this based on database metadata + return new String[0]; + } + + @Override + public Class[] getFieldTypes() { + // TODO: Implement this based on database metadata + return new Class[0]; + } + + @Override + public void setConfig(Config config) { + if(config != null) { + this.config = config; + } + } + + @Override + public Config getConfig() { + return config; + } + + @Override + public CheckResult checkConfig() { + List requiredOptions = Arrays.asList(URL, TABLE, USER); + + List nonExistsOptions = new ArrayList<>(); + requiredOptions.forEach(x->{ + if(!config.has(x)){ + nonExistsOptions.add(x); + } + }); + + if (!nonExistsOptions.isEmpty()) { + return new CheckResult( + false, + "please specify " + String.join(",", nonExistsOptions.stream() + .map(option -> "[" + option + "]") + .collect(Collectors.toList())) + " as non-empty string"); + } else { + return new CheckResult(true, ""); + } + } + + @Override + public void prepare(RuntimeEnvironment env) throws Exception { + // TODO Auto-generated method stub + throw new UnsupportedOperationException("Unimplemented method 'prepare'"); + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/pom.xml new file mode 100644 index 000000000..658f8deb6 --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/pom.xml @@ -0,0 +1,51 @@ + + + + + datavines-engine-flink + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink-transform + + + + io.datavines + datavines-engine-flink-core + ${project.version} + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink.version} + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + + + diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/src/main/java/io/datavines/engine/flink/transform/FlinkSqlTransform.java b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/src/main/java/io/datavines/engine/flink/transform/FlinkSqlTransform.java new file mode 100644 index 000000000..28aa56ccc --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/datavines-engine-flink-transform/src/main/java/io/datavines/engine/flink/transform/FlinkSqlTransform.java @@ -0,0 +1,87 @@ +/* + * 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.transform; + +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 io.datavines.common.config.Config; +import io.datavines.common.config.CheckResult; +import io.datavines.engine.api.env.RuntimeEnvironment; +import io.datavines.engine.flink.api.FlinkRuntimeEnvironment; +import io.datavines.engine.flink.api.stream.FlinkStreamTransform; + +import static io.datavines.common.ConfigConstants.SQL; + +public class FlinkSqlTransform implements FlinkStreamTransform { + + private Config config = new Config(); + + @Override + public DataStream process(DataStream dataStream, FlinkRuntimeEnvironment environment) { + StreamTableEnvironment tableEnv = environment.getTableEnv(); + + // Register input as table + tableEnv.createTemporaryView("input_table", dataStream); + + // Execute SQL transformation + Table resultTable = tableEnv.sqlQuery(config.getString(SQL)); + + // Convert back to DataStream + return tableEnv.toDataStream(resultTable); + } + + @Override + public void setConfig(Config config) { + if(config != null) { + this.config = config; + } + } + + @Override + public Config getConfig() { + return config; + } + + @Override + public CheckResult checkConfig() { + if (config.has(SQL)) { + return new CheckResult(true, ""); + } else { + return new CheckResult(false, "please specify [sql]"); + } + } + + @Override + public void prepare(RuntimeEnvironment env) throws Exception { + // No preparation needed + } + + @Override + public String[] getOutputFieldNames() { + // The output field names will be determined by the SQL query result + return new String[0]; + } + + @Override + public Class[] getOutputFieldTypes() { + // The output field types will be determined by the SQL query result + return new Class[0]; + } +} diff --git a/datavines-engine/datavines-engine-plugins/datavines-engine-flink/pom.xml b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/pom.xml new file mode 100644 index 000000000..0209d987a --- /dev/null +++ b/datavines-engine/datavines-engine-plugins/datavines-engine-flink/pom.xml @@ -0,0 +1,114 @@ + + + + + datavines-engine-plugins + io.datavines + 1.0.0-SNAPSHOT + + 4.0.0 + + datavines-engine-flink + pom + + + datavines-engine-flink-api + datavines-engine-flink-core + datavines-engine-flink-config + datavines-engine-flink-transform + datavines-engine-flink-jdbc + datavines-engine-flink-executor + + + + 1.13.6 + 2.11 + + + + + central + aliyun maven + https://maven.aliyun.com/repository/public + + true + + + true + + + + apache + apache maven + https://repository.apache.org/content/repositories/releases/ + + true + + + false + + + + + + + io.datavines + datavines-engine-api + ${project.version} + + + + org.apache.flink + flink-java + ${flink.version} + provided + + + + org.apache.flink + flink-streaming-java_${scala.binary.version} + ${flink.version} + provided + + + + org.apache.flink + flink-clients_${scala.binary.version} + ${flink.version} + provided + + + + org.apache.flink + flink-table-api-java-bridge_${scala.binary.version} + ${flink.version} + provided + + + + org.apache.flink + flink-table-planner_${scala.binary.version} + ${flink.version} + provided + + + diff --git a/datavines-engine/datavines-engine-plugins/pom.xml b/datavines-engine/datavines-engine-plugins/pom.xml index d27ae2fa5..57112b5f0 100644 --- a/datavines-engine/datavines-engine-plugins/pom.xml +++ b/datavines-engine/datavines-engine-plugins/pom.xml @@ -1,21 +1,21 @@ datavines-engine-spark + datavines-engine-flink datavines-engine-local datavines-engine-livy diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-daily-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-daily-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index 76e0227c9..c3217560a 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-daily-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-daily-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_daily_avg=io.datavines.metric.expected.plugin.DailyAvg spark_daily_avg=io.datavines.metric.expected.plugin.DailyAvg -livy_daily_avg=io.datavines.metric.expected.plugin.DailyAvg \ No newline at end of file +livy_daily_avg=io.datavines.metric.expected.plugin.DailyAvg +flink_daily_avg=io.datavines.metric.expected.plugin.DailyAvg diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-fix/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-fix/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index b53917884..bce9667bb 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-fix/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-fix/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_fix_value=io.datavines.metric.expected.plugin.FixValue spark_fix_value=io.datavines.metric.expected.plugin.FixValue -livy_fix_value=io.datavines.metric.expected.plugin.FixValue \ No newline at end of file +livy_fix_value=io.datavines.metric.expected.plugin.FixValue +flink_fix_value=io.datavines.metric.expected.plugin.FixValue \ No newline at end of file diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/java/io/datavines/metric/expected/plugin/Last30DayAvg.java b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/java/io/datavines/metric/expected/plugin/Last30DayAvg.java index bce4640e1..56d0b7f71 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/java/io/datavines/metric/expected/plugin/Last30DayAvg.java +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/java/io/datavines/metric/expected/plugin/Last30DayAvg.java @@ -1,71 +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 io.datavines.metric.expected.plugin; - -import io.datavines.common.exception.DataVinesException; -import java.util.Map; - -import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; - -public class Last30DayAvg extends AbstractExpectedValue { - - @Override - public String getName() { - return "last_30d_avg"; - } - - @Override - public String getZhName() { - return "最近30天均值"; - } - - @Override - public String getKey(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - return "expected_value_" + uniqueKey; - } - - @Override - public String getExecuteSql(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - String engineType = inputParameter.get("engine_type"); - switch (engineType){ - case "spark": - return getConnectorFactory("spark").getMetricScript().last30DayAvg(uniqueKey); - case "local": - return getConnectorFactory(inputParameter).getMetricScript().last30DayAvg(uniqueKey); - default: - throw new DataVinesException(String.format("engine type %s is not supported", engineType)); - } - } - - @Override - public String getOutputTable(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - return "last_30d_" + uniqueKey; - } - - @Override - public boolean isNeedDefaultDatasource() { - return true; - } - - @Override - public void prepare(Map config) { - - } -} +/* + * 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.metric.expected.plugin; + +import io.datavines.common.exception.DataVinesException; +import java.util.Map; + +import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; + +public class Last30DayAvg extends AbstractExpectedValue { + + @Override + public String getName() { + return "last_30d_avg"; + } + + @Override + public String getZhName() { + return "最近30天均值"; + } + + @Override + public String getKey(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + return "expected_value_" + uniqueKey; + } + + @Override + public String getExecuteSql(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + String engineType = inputParameter.get("engine_type"); + switch (engineType){ + case "spark": + return getConnectorFactory("spark").getMetricScript().last30DayAvg(uniqueKey); + case "local": + return getConnectorFactory(inputParameter).getMetricScript().last30DayAvg(uniqueKey); + default: + throw new DataVinesException(String.format("engine type %s is not supported", engineType)); + } + } + + @Override + public String getOutputTable(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + return "last_30d_" + uniqueKey; + } + + @Override + public boolean isNeedDefaultDatasource() { + return true; + } + + @Override + public void prepare(Map config) { + + } +} diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index 44995d759..0a9716fd9 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last30day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_last_30d_avg=io.datavines.metric.expected.plugin.Last30DayAvg spark_last_30d_avg=io.datavines.metric.expected.plugin.Last30DayAvg -livy_last_30d_avg=io.datavines.metric.expected.plugin.Last30DayAvg \ No newline at end of file +livy_last_30d_avg=io.datavines.metric.expected.plugin.Last30DayAvg +flink_last_30d_avg=io.datavines.metric.expected.plugin.Last30DayAvg diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last7day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last7day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index 88c272e68..4c4a340ae 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last7day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-last7day-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_last_7d_avg=io.datavines.metric.expected.plugin.Last7DayAvg spark_last_7d_avg=io.datavines.metric.expected.plugin.Last7DayAvg -livy_last_7d_avg=io.datavines.metric.expected.plugin.Last7DayAvg \ No newline at end of file +livy_last_7d_avg=io.datavines.metric.expected.plugin.Last7DayAvg +flink_last_7d_avg=io.datavines.metric.expected.plugin.Last7DayAvg diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-monthly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-monthly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index 6fc8acdf6..ee2481e4c 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-monthly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-monthly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_monthly_avg=io.datavines.metric.expected.plugin.MonthlyAvg spark_monthly_avg=io.datavines.metric.expected.plugin.MonthlyAvg -livy_monthly_avg=io.datavines.metric.expected.plugin.MonthlyAvg \ No newline at end of file +livy_monthly_avg=io.datavines.metric.expected.plugin.MonthlyAvg +flink_monthly_avg=io.datavines.metric.expected.plugin.MonthlyAvg diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-none/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-none/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index c23fce078..4134bd927 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-none/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-none/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_none=io.datavines.metric.expected.plugin.None spark_none=io.datavines.metric.expected.plugin.SparkNone -livy_none=io.datavines.metric.expected.plugin.SparkNone \ No newline at end of file +livy_none=io.datavines.metric.expected.plugin.SparkNone +flink_none=io.datavines.metric.expected.plugin.None diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index e0db566b2..f84be9653 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_table_total_rows=io.datavines.metric.expected.plugin.TableTotalRows spark_table_total_rows=io.datavines.metric.expected.plugin.TableTotalRows -livy_table_total_rows=io.datavines.metric.expected.plugin.TableTotalRows \ No newline at end of file +livy_table_total_rows=io.datavines.metric.expected.plugin.TableTotalRows +flink_table_total_rows=io.datavines.metric.expected.plugin.TableTotalRows \ No newline at end of file diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-target-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-target-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index f93c9dcc6..b105fbcda 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-target-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-target-table-rows/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_target_table_total_rows=io.datavines.metric.expected.plugin.TargetTableTotalRows spark_target_table_total_rows=io.datavines.metric.expected.plugin.TargetTableTotalRows -livy_target_table_total_rows=io.datavines.metric.expected.plugin.TargetTableTotalRows \ No newline at end of file +livy_target_table_total_rows=io.datavines.metric.expected.plugin.TargetTableTotalRows +flink_target_table_total_rows=io.datavines.metric.expected.plugin.TargetTableTotalRows \ No newline at end of file diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/java/io/datavines/metric/expected/plugin/WeeklyAvg.java b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/java/io/datavines/metric/expected/plugin/WeeklyAvg.java index 76891c5a3..f6eef774d 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/java/io/datavines/metric/expected/plugin/WeeklyAvg.java +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/java/io/datavines/metric/expected/plugin/WeeklyAvg.java @@ -1,72 +1,72 @@ -/* - * 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.metric.expected.plugin; - -import io.datavines.common.exception.DataVinesException; - -import java.util.Map; - -import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; - -public class WeeklyAvg extends AbstractExpectedValue { - - @Override - public String getName() { - return "weekly_avg"; - } - - @Override - public String getKey(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - return "expected_value_" + uniqueKey; - } - - @Override - public String getZhName() { - return "周均值"; - } - - @Override - public String getExecuteSql(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - String engineType = inputParameter.get("engine_type"); - switch (engineType){ - case "spark": - return getConnectorFactory("spark").getMetricScript().weeklyAvg(uniqueKey); - case "local": - return getConnectorFactory(inputParameter).getMetricScript().weeklyAvg(uniqueKey); - default: - throw new DataVinesException(String.format("engine type %s is not supported", engineType)); - } - } - - @Override - public String getOutputTable(Map inputParameter) { - String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); - return "weekly_range_" + uniqueKey; - } - - @Override - public boolean isNeedDefaultDatasource() { - return true; - } - - @Override - public void prepare(Map config) { - - } -} +/* + * 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.metric.expected.plugin; + +import io.datavines.common.exception.DataVinesException; + +import java.util.Map; + +import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; + +public class WeeklyAvg extends AbstractExpectedValue { + + @Override + public String getName() { + return "weekly_avg"; + } + + @Override + public String getKey(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + return "expected_value_" + uniqueKey; + } + + @Override + public String getZhName() { + return "周均值"; + } + + @Override + public String getExecuteSql(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + String engineType = inputParameter.get("engine_type"); + switch (engineType){ + case "spark": + return getConnectorFactory("spark").getMetricScript().weeklyAvg(uniqueKey); + case "local": + return getConnectorFactory(inputParameter).getMetricScript().weeklyAvg(uniqueKey); + default: + throw new DataVinesException(String.format("engine type %s is not supported", engineType)); + } + } + + @Override + public String getOutputTable(Map inputParameter) { + String uniqueKey = inputParameter.get(METRIC_UNIQUE_KEY); + return "weekly_range_" + uniqueKey; + } + + @Override + public boolean isNeedDefaultDatasource() { + return true; + } + + @Override + public void prepare(Map config) { + + } +} diff --git a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue index 7e8ff7a04..eef2d6152 100644 --- a/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue +++ b/datavines-metric/datavines-metric-expected-plugins/datavines-metric-expected-weekly-avg/src/main/resources/META-INF/plugins/io.datavines.metric.api.ExpectedValue @@ -1,3 +1,4 @@ local_weekly_avg=io.datavines.metric.expected.plugin.WeeklyAvg spark_weekly_avg=io.datavines.metric.expected.plugin.WeeklyAvg -livy_weekly_avg=io.datavines.metric.expected.plugin.WeeklyAvg \ No newline at end of file +livy_weekly_avg=io.datavines.metric.expected.plugin.WeeklyAvg +flink_weekly_avg=io.datavines.metric.expected.plugin.WeeklyAvg \ No newline at end of file diff --git a/datavines-metric/datavines-metric-plugins/datavines-metric-custom-aggregate-sql/src/main/java/io/datavines/metric/plugin/CustomAggregateSql.java b/datavines-metric/datavines-metric-plugins/datavines-metric-custom-aggregate-sql/src/main/java/io/datavines/metric/plugin/CustomAggregateSql.java index 809cb1644..092e7c213 100644 --- a/datavines-metric/datavines-metric-plugins/datavines-metric-custom-aggregate-sql/src/main/java/io/datavines/metric/plugin/CustomAggregateSql.java +++ b/datavines-metric/datavines-metric-plugins/datavines-metric-custom-aggregate-sql/src/main/java/io/datavines/metric/plugin/CustomAggregateSql.java @@ -1,113 +1,113 @@ -/* - * 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.metric.plugin; - -import java.util.*; - -import io.datavines.common.config.CheckResult; -import io.datavines.common.config.ConfigChecker; -import io.datavines.common.entity.ExecuteSql; -import io.datavines.common.enums.DataVinesDataType; -import io.datavines.common.utils.StringUtils; -import io.datavines.metric.api.ConfigItem; -import io.datavines.metric.api.MetricDimension; -import io.datavines.metric.api.MetricType; -import io.datavines.metric.api.SqlMetric; - -import static io.datavines.common.CommonConstants.TABLE; -import static io.datavines.common.ConfigConstants.*; -import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; - -public class CustomAggregateSql implements SqlMetric { - - private final Set requiredOptions = new HashSet<>(); - - private final HashMap configMap = new HashMap<>(); - - public CustomAggregateSql() { - configMap.put("table",new ConfigItem("table", "表名", "table")); - configMap.put("actual_aggregate_sql", new ConfigItem("actual_aggregate_sql","自定义聚合SQL","actual_aggregate_sql")); - configMap.put("filter",new ConfigItem("filter", "过滤条件", "filter")); - - requiredOptions.add("actual_aggregate_sql"); - requiredOptions.add("table"); - } - - @Override - public String getName() { - return "custom_aggregate_sql"; - } - - @Override - public String getZhName() { - return "自定义聚合SQL"; - } - - @Override - public MetricDimension getDimension() { - return MetricDimension.ACCURACY; - } - - @Override - public MetricType getType() { - return MetricType.SINGLE_TABLE; - } - - @Override - public boolean isInvalidateItemsCanOutput() { - return false; - } - - @Override - public CheckResult validateConfig(Map config) { - return ConfigChecker.checkConfig(config, requiredOptions); - } - - @Override - public void prepare(Map config) { - - } - - @Override - public Map getConfigMap() { - return configMap; - } - - @Override - public ExecuteSql getInvalidateItems(Map inputParameter) { - return null; - } - - @Override - public ExecuteSql getActualValue(Map inputParameter) { - inputParameter.put(ACTUAL_TABLE, inputParameter.get(TABLE)); - String actualAggregateSql = inputParameter.get(ACTUAL_AGGREGATE_SQL); - if (StringUtils.isNotEmpty(actualAggregateSql)) { - if (actualAggregateSql.contains("as actual_value")) { - actualAggregateSql = actualAggregateSql.replace("as actual_value", "as actual_value_" + inputParameter.get(METRIC_UNIQUE_KEY)); - } else if (actualAggregateSql.contains("AS actual_value")) { - actualAggregateSql = actualAggregateSql.replace("AS actual_value", "as actual_value_" + inputParameter.get(METRIC_UNIQUE_KEY)); - } - } - return new ExecuteSql(actualAggregateSql, inputParameter.get(TABLE)); - } - - @Override - public List suitableType() { - return Collections.emptyList(); - } -} +/* + * 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.metric.plugin; + +import java.util.*; + +import io.datavines.common.config.CheckResult; +import io.datavines.common.config.ConfigChecker; +import io.datavines.common.entity.ExecuteSql; +import io.datavines.common.enums.DataVinesDataType; +import io.datavines.common.utils.StringUtils; +import io.datavines.metric.api.ConfigItem; +import io.datavines.metric.api.MetricDimension; +import io.datavines.metric.api.MetricType; +import io.datavines.metric.api.SqlMetric; + +import static io.datavines.common.CommonConstants.TABLE; +import static io.datavines.common.ConfigConstants.*; +import static io.datavines.common.ConfigConstants.METRIC_UNIQUE_KEY; + +public class CustomAggregateSql implements SqlMetric { + + private final Set requiredOptions = new HashSet<>(); + + private final HashMap configMap = new HashMap<>(); + + public CustomAggregateSql() { + configMap.put("table",new ConfigItem("table", "表名", "table")); + configMap.put("actual_aggregate_sql", new ConfigItem("actual_aggregate_sql","自定义聚合SQL","actual_aggregate_sql")); + configMap.put("filter",new ConfigItem("filter", "过滤条件", "filter")); + + requiredOptions.add("actual_aggregate_sql"); + requiredOptions.add("table"); + } + + @Override + public String getName() { + return "custom_aggregate_sql"; + } + + @Override + public String getZhName() { + return "自定义聚合SQL"; + } + + @Override + public MetricDimension getDimension() { + return MetricDimension.ACCURACY; + } + + @Override + public MetricType getType() { + return MetricType.SINGLE_TABLE; + } + + @Override + public boolean isInvalidateItemsCanOutput() { + return false; + } + + @Override + public CheckResult validateConfig(Map config) { + return ConfigChecker.checkConfig(config, requiredOptions); + } + + @Override + public void prepare(Map config) { + + } + + @Override + public Map getConfigMap() { + return configMap; + } + + @Override + public ExecuteSql getInvalidateItems(Map inputParameter) { + return null; + } + + @Override + public ExecuteSql getActualValue(Map inputParameter) { + inputParameter.put(ACTUAL_TABLE, inputParameter.get(TABLE)); + String actualAggregateSql = inputParameter.get(ACTUAL_AGGREGATE_SQL); + if (StringUtils.isNotEmpty(actualAggregateSql)) { + if (actualAggregateSql.contains("as actual_value")) { + actualAggregateSql = actualAggregateSql.replace("as actual_value", "as actual_value_" + inputParameter.get(METRIC_UNIQUE_KEY)); + } else if (actualAggregateSql.contains("AS actual_value")) { + actualAggregateSql = actualAggregateSql.replace("AS actual_value", "as actual_value_" + inputParameter.get(METRIC_UNIQUE_KEY)); + } + } + return new ExecuteSql(actualAggregateSql, inputParameter.get(TABLE)); + } + + @Override + public List suitableType() { + return Collections.emptyList(); + } +} diff --git a/datavines-registry/datavines-registry-plugins/datavines-registry-mysql/src/main/java/io/datavines/registry/plugin/MysqlServerStateManager.java b/datavines-registry/datavines-registry-plugins/datavines-registry-mysql/src/main/java/io/datavines/registry/plugin/MysqlServerStateManager.java index 3908bc321..9b4eaba47 100644 --- a/datavines-registry/datavines-registry-plugins/datavines-registry-mysql/src/main/java/io/datavines/registry/plugin/MysqlServerStateManager.java +++ b/datavines-registry/datavines-registry-plugins/datavines-registry-mysql/src/main/java/io/datavines/registry/plugin/MysqlServerStateManager.java @@ -254,4 +254,4 @@ public void close() throws SQLException { connection.close(); } } -} +} \ No newline at end of file diff --git a/datavines-server/pom.xml b/datavines-server/pom.xml index 89d9a34b4..90607d9c7 100644 --- a/datavines-server/pom.xml +++ b/datavines-server/pom.xml @@ -1,21 +1,21 @@ ${project.version} + + io.datavines + datavines-engine-flink-executor + ${project.version} + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-yarn-common + + + + + + io.datavines + datavines-engine-flink-core + ${project.version} + + io.datavines datavines-connector-all diff --git a/datavines-server/src/main/java/io/datavines/server/api/config/WebMvcConfig.java b/datavines-server/src/main/java/io/datavines/server/api/config/WebMvcConfig.java index d522c1734..9c3a98e2e 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/config/WebMvcConfig.java +++ b/datavines-server/src/main/java/io/datavines/server/api/config/WebMvcConfig.java @@ -49,13 +49,17 @@ public void addInterceptors(InterceptorRegistry registry) { @Override public void addResourceHandlers(ResourceHandlerRegistry registry) { + registry.addResourceHandler("swagger-ui.html") + .addResourceLocations("classpath:/META-INF/resources/"); + + registry.addResourceHandler("/webjars/**") + .addResourceLocations("classpath:/META-INF/resources/webjars/"); registry.addResourceHandler("/**") .addResourceLocations("classpath:/META-INF/resources/") .addResourceLocations("classpath:/static/") .addResourceLocations("classpath:/static/templates") - .addResourceLocations("classpath:/public/") - ; + .addResourceLocations("classpath:/public/"); } @Override diff --git a/datavines-server/src/main/java/io/datavines/server/api/controller/JobExecutionController.java b/datavines-server/src/main/java/io/datavines/server/api/controller/JobExecutionController.java index 79ad90002..286d9aef2 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/controller/JobExecutionController.java +++ b/datavines-server/src/main/java/io/datavines/server/api/controller/JobExecutionController.java @@ -1,123 +1,123 @@ -/* - * 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.server.api.controller; - -import io.datavines.core.aop.RefreshToken; -import io.datavines.core.constant.DataVinesConstants; -import io.datavines.core.exception.DataVinesServerException; -import io.datavines.common.entity.job.SubmitJob; -import io.datavines.server.api.dto.bo.job.JobExecutionDashboardParam; -import io.datavines.server.api.dto.bo.job.JobExecutionPageParam; -import io.datavines.server.api.dto.vo.JobExecutionResultVO; -import io.datavines.server.repository.entity.JobExecution; -import io.datavines.server.repository.service.JobExecutionErrorDataService; -import io.datavines.server.repository.service.JobExecutionResultService; -import io.datavines.server.repository.service.JobExecutionService; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import lombok.extern.slf4j.Slf4j; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.http.MediaType; -import org.springframework.web.bind.annotation.*; - -import javax.validation.Valid; - -@Slf4j -@Api(value = "job", tags = "job", produces = MediaType.APPLICATION_JSON_VALUE) -@RestController -@RequestMapping(value = DataVinesConstants.BASE_API_PATH + "/job/execution", produces = MediaType.APPLICATION_JSON_VALUE) -@RefreshToken -public class JobExecutionController { - - @Autowired - private JobExecutionService jobExecutionService; - - @Autowired - private JobExecutionResultService jobExecutionResultService; - - @Autowired - private JobExecutionErrorDataService jobExecutionErrorDataService; - - @ApiOperation(value = "submit external data quality job", response = Long.class) - @PostMapping(value = "/submit/data-quality", consumes = MediaType.APPLICATION_JSON_VALUE) - public Object submitDataQualityJob(@Valid @RequestBody SubmitJob submitJob) throws DataVinesServerException { - return jobExecutionService.submitJob(submitJob); - } - - @ApiOperation(value = "submit external data reconciliation job", response = Long.class) - @PostMapping(value = "/submit/data-reconciliation", consumes = MediaType.APPLICATION_JSON_VALUE) - public Object submitDataReconJob(@Valid @RequestBody SubmitJob submitJob) throws DataVinesServerException { - return jobExecutionService.submitJob(submitJob); - } - - @ApiOperation(value = "kill job", response = Long.class) - @DeleteMapping(value = "/kill/{executionId}") - public Object killTask(@PathVariable("executionId") Long executionId) { - return jobExecutionService.killJob(executionId); - } - - @ApiOperation(value = "get job execution status", response = String.class) - @GetMapping(value = "/status/{executionId}") - public Object getTaskStatus(@PathVariable("executionId") Long executionId) { - return jobExecutionService.getById(executionId).getStatus().getDescription(); - } - - @ApiOperation(value = "get job execution list by job id", response = JobExecution.class, responseContainer = "list") - @GetMapping(value = "/list/{jobId}") - public Object getJobExecutionListByJobId(@PathVariable("jobId") Long jobId) { - return jobExecutionService.listByJobId(jobId); - } - - @Deprecated - @ApiOperation(value = "get job execution result", response = JobExecutionResultVO.class) - @GetMapping(value = "/result/{executionId}") - public Object getJobExecutionResultInfo(@PathVariable("executionId") Long executionId) { - return jobExecutionResultService.getResultVOByJobExecutionId(executionId); - } - - @ApiOperation(value = "get job execution result", response = JobExecutionResultVO.class) - @GetMapping(value = "/list/result/{executionId}") - public Object getJobExecutionResultInfoList(@PathVariable("executionId") Long executionId) { - return jobExecutionResultService.getResultVOListByJobExecutionId(executionId); - } - - @ApiOperation(value = "get job execution page", response = JobExecutionResultVO.class, responseContainer = "page") - @PostMapping(value = "/page") - public Object page(@Valid @RequestBody JobExecutionPageParam jobExecutionPageParam) { - return jobExecutionService.getJobExecutionPage(jobExecutionPageParam); - } - - @ApiOperation(value = "get job execution error data page", response = Object.class, responseContainer = "page") - @GetMapping(value = "/errorDataPage") - public Object readErrorDataPage(@RequestParam("taskId") Long taskId, - @RequestParam("pageNumber") Integer pageNumber, - @RequestParam("pageSize") Integer pageSize){ - return jobExecutionErrorDataService.readErrorDataPage(taskId, pageNumber, pageSize); - } - - @ApiOperation(value = "get job execution agg pie", response = JobExecutionResultVO.class) - @PostMapping(value = "/agg-pie") - public Object getExecutionAggPie(@Valid @RequestBody JobExecutionDashboardParam dashboardParam) { - return jobExecutionService.getJobExecutionAggPie(dashboardParam); - } - - @ApiOperation(value = "get job execution trend bar", response = JobExecutionResultVO.class) - @PostMapping(value = "/trend-bar") - public Object getExecutionTrendBar(@Valid @RequestBody JobExecutionDashboardParam dashboardParam) { - return jobExecutionService.getJobExecutionTrendBar(dashboardParam); - } -} +/* + * 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.server.api.controller; + +import io.datavines.core.aop.RefreshToken; +import io.datavines.core.constant.DataVinesConstants; +import io.datavines.core.exception.DataVinesServerException; +import io.datavines.common.entity.job.SubmitJob; +import io.datavines.server.api.dto.bo.job.JobExecutionDashboardParam; +import io.datavines.server.api.dto.bo.job.JobExecutionPageParam; +import io.datavines.server.api.dto.vo.JobExecutionResultVO; +import io.datavines.server.repository.entity.JobExecution; +import io.datavines.server.repository.service.JobExecutionErrorDataService; +import io.datavines.server.repository.service.JobExecutionResultService; +import io.datavines.server.repository.service.JobExecutionService; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.MediaType; +import org.springframework.web.bind.annotation.*; + +import javax.validation.Valid; + +@Slf4j +@Api(value = "job", tags = "job", produces = MediaType.APPLICATION_JSON_VALUE) +@RestController +@RequestMapping(value = DataVinesConstants.BASE_API_PATH + "/job/execution", produces = MediaType.APPLICATION_JSON_VALUE) +@RefreshToken +public class JobExecutionController { + + @Autowired + private JobExecutionService jobExecutionService; + + @Autowired + private JobExecutionResultService jobExecutionResultService; + + @Autowired + private JobExecutionErrorDataService jobExecutionErrorDataService; + + @ApiOperation(value = "submit external data quality job", response = Long.class) + @PostMapping(value = "/submit/data-quality", consumes = MediaType.APPLICATION_JSON_VALUE) + public Object submitDataQualityJob(@Valid @RequestBody SubmitJob submitJob) throws DataVinesServerException { + return jobExecutionService.submitJob(submitJob); + } + + @ApiOperation(value = "submit external data reconciliation job", response = Long.class) + @PostMapping(value = "/submit/data-reconciliation", consumes = MediaType.APPLICATION_JSON_VALUE) + public Object submitDataReconJob(@Valid @RequestBody SubmitJob submitJob) throws DataVinesServerException { + return jobExecutionService.submitJob(submitJob); + } + + @ApiOperation(value = "kill job", response = Long.class) + @DeleteMapping(value = "/kill/{executionId}") + public Object killTask(@PathVariable("executionId") Long executionId) { + return jobExecutionService.killJob(executionId); + } + + @ApiOperation(value = "get job execution status", response = String.class) + @GetMapping(value = "/status/{executionId}") + public Object getTaskStatus(@PathVariable("executionId") Long executionId) { + return jobExecutionService.getById(executionId).getStatus().getDescription(); + } + + @ApiOperation(value = "get job execution list by job id", response = JobExecution.class, responseContainer = "list") + @GetMapping(value = "/list/{jobId}") + public Object getJobExecutionListByJobId(@PathVariable("jobId") Long jobId) { + return jobExecutionService.listByJobId(jobId); + } + + @Deprecated + @ApiOperation(value = "get job execution result", response = JobExecutionResultVO.class) + @GetMapping(value = "/result/{executionId}") + public Object getJobExecutionResultInfo(@PathVariable("executionId") Long executionId) { + return jobExecutionResultService.getResultVOByJobExecutionId(executionId); + } + + @ApiOperation(value = "get job execution result", response = JobExecutionResultVO.class) + @GetMapping(value = "/list/result/{executionId}") + public Object getJobExecutionResultInfoList(@PathVariable("executionId") Long executionId) { + return jobExecutionResultService.getResultVOListByJobExecutionId(executionId); + } + + @ApiOperation(value = "get job execution page", response = JobExecutionResultVO.class, responseContainer = "page") + @PostMapping(value = "/page") + public Object page(@Valid @RequestBody JobExecutionPageParam jobExecutionPageParam) { + return jobExecutionService.getJobExecutionPage(jobExecutionPageParam); + } + + @ApiOperation(value = "get job execution error data page", response = Object.class, responseContainer = "page") + @GetMapping(value = "/errorDataPage") + public Object readErrorDataPage(@RequestParam("taskId") Long taskId, + @RequestParam("pageNumber") Integer pageNumber, + @RequestParam("pageSize") Integer pageSize){ + return jobExecutionErrorDataService.readErrorDataPage(taskId, pageNumber, pageSize); + } + + @ApiOperation(value = "get job execution agg pie", response = JobExecutionResultVO.class) + @PostMapping(value = "/agg-pie") + public Object getExecutionAggPie(@Valid @RequestBody JobExecutionDashboardParam dashboardParam) { + return jobExecutionService.getJobExecutionAggPie(dashboardParam); + } + + @ApiOperation(value = "get job execution trend bar", response = JobExecutionResultVO.class) + @PostMapping(value = "/trend-bar") + public Object getExecutionTrendBar(@Valid @RequestBody JobExecutionDashboardParam dashboardParam) { + return jobExecutionService.getJobExecutionTrendBar(dashboardParam); + } +} diff --git a/datavines-server/src/main/java/io/datavines/server/api/controller/JobQualityReportController.java b/datavines-server/src/main/java/io/datavines/server/api/controller/JobQualityReportController.java index 24bff4b23..a484e26f4 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/controller/JobQualityReportController.java +++ b/datavines-server/src/main/java/io/datavines/server/api/controller/JobQualityReportController.java @@ -1,83 +1,83 @@ -/* - * 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.server.api.controller; - -import io.datavines.common.entity.job.SubmitJob; -import io.datavines.core.aop.RefreshToken; -import io.datavines.core.constant.DataVinesConstants; -import io.datavines.core.exception.DataVinesServerException; -import io.datavines.server.api.dto.bo.job.JobExecutionDashboardParam; -import io.datavines.server.api.dto.bo.job.JobExecutionPageParam; -import io.datavines.server.api.dto.bo.job.JobQualityReportDashboardParam; -import io.datavines.server.api.dto.vo.JobExecutionResultVO; -import io.datavines.server.repository.entity.JobExecution; -import io.datavines.server.repository.service.JobExecutionErrorDataService; -import io.datavines.server.repository.service.JobExecutionResultService; -import io.datavines.server.repository.service.JobExecutionService; -import io.datavines.server.repository.service.JobQualityReportService; -import io.swagger.annotations.Api; -import io.swagger.annotations.ApiOperation; -import lombok.extern.slf4j.Slf4j; -import org.springframework.beans.factory.annotation.Autowired; -import org.springframework.http.MediaType; -import org.springframework.web.bind.annotation.*; - -import javax.validation.Valid; - -@Slf4j -@Api(value = "job-quality-report", tags = "job-quality-report", produces = MediaType.APPLICATION_JSON_VALUE) -@RestController -@RequestMapping(value = DataVinesConstants.BASE_API_PATH + "/job/quality-report", produces = MediaType.APPLICATION_JSON_VALUE) -@RefreshToken -public class JobQualityReportController { - - @Autowired - private JobExecutionService jobExecutionService; - - @Autowired - private JobExecutionResultService jobExecutionResultService; - - @Autowired - private JobExecutionErrorDataService jobExecutionErrorDataService; - - @Autowired - private JobQualityReportService jobQualityReportService; - - @ApiOperation(value = "get job quality report page", response = JobExecutionResultVO.class, responseContainer = "page") - @PostMapping(value = "/page") - public Object page(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { - return jobQualityReportService.getQualityReportPage(dashboardParam); - } - - @ApiOperation(value = "get job quality report page", response = JobExecutionResultVO.class, responseContainer = "list") - @GetMapping(value = "/listColumnExecution") - public Object listColumnExecution(@RequestParam Long reportId) { - return jobQualityReportService.listColumnExecution(reportId); - } - - @ApiOperation(value = "get job quality report score", response = JobExecutionResultVO.class) - @PostMapping(value = "/score") - public Object getScoreByCondition(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { - return jobQualityReportService.getScoreByCondition(dashboardParam); - } - - @ApiOperation(value = "get job quality report score trend", response = JobExecutionResultVO.class) - @PostMapping(value = "/score-trend") - public Object getScoreTrendByCondition(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { - return jobQualityReportService.getScoreTrendByCondition(dashboardParam); - } -} +/* + * 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.server.api.controller; + +import io.datavines.common.entity.job.SubmitJob; +import io.datavines.core.aop.RefreshToken; +import io.datavines.core.constant.DataVinesConstants; +import io.datavines.core.exception.DataVinesServerException; +import io.datavines.server.api.dto.bo.job.JobExecutionDashboardParam; +import io.datavines.server.api.dto.bo.job.JobExecutionPageParam; +import io.datavines.server.api.dto.bo.job.JobQualityReportDashboardParam; +import io.datavines.server.api.dto.vo.JobExecutionResultVO; +import io.datavines.server.repository.entity.JobExecution; +import io.datavines.server.repository.service.JobExecutionErrorDataService; +import io.datavines.server.repository.service.JobExecutionResultService; +import io.datavines.server.repository.service.JobExecutionService; +import io.datavines.server.repository.service.JobQualityReportService; +import io.swagger.annotations.Api; +import io.swagger.annotations.ApiOperation; +import lombok.extern.slf4j.Slf4j; +import org.springframework.beans.factory.annotation.Autowired; +import org.springframework.http.MediaType; +import org.springframework.web.bind.annotation.*; + +import javax.validation.Valid; + +@Slf4j +@Api(value = "job-quality-report", tags = "job-quality-report", produces = MediaType.APPLICATION_JSON_VALUE) +@RestController +@RequestMapping(value = DataVinesConstants.BASE_API_PATH + "/job/quality-report", produces = MediaType.APPLICATION_JSON_VALUE) +@RefreshToken +public class JobQualityReportController { + + @Autowired + private JobExecutionService jobExecutionService; + + @Autowired + private JobExecutionResultService jobExecutionResultService; + + @Autowired + private JobExecutionErrorDataService jobExecutionErrorDataService; + + @Autowired + private JobQualityReportService jobQualityReportService; + + @ApiOperation(value = "get job quality report page", response = JobExecutionResultVO.class, responseContainer = "page") + @PostMapping(value = "/page") + public Object page(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { + return jobQualityReportService.getQualityReportPage(dashboardParam); + } + + @ApiOperation(value = "get job quality report page", response = JobExecutionResultVO.class, responseContainer = "list") + @GetMapping(value = "/listColumnExecution") + public Object listColumnExecution(@RequestParam Long reportId) { + return jobQualityReportService.listColumnExecution(reportId); + } + + @ApiOperation(value = "get job quality report score", response = JobExecutionResultVO.class) + @PostMapping(value = "/score") + public Object getScoreByCondition(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { + return jobQualityReportService.getScoreByCondition(dashboardParam); + } + + @ApiOperation(value = "get job quality report score trend", response = JobExecutionResultVO.class) + @PostMapping(value = "/score-trend") + public Object getScoreTrendByCondition(@Valid @RequestBody JobQualityReportDashboardParam dashboardParam) { + return jobQualityReportService.getScoreTrendByCondition(dashboardParam); + } +} diff --git a/datavines-server/src/main/java/io/datavines/server/api/controller/MetricController.java b/datavines-server/src/main/java/io/datavines/server/api/controller/MetricController.java index 197bdf6c8..808cf5139 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/controller/MetricController.java +++ b/datavines-server/src/main/java/io/datavines/server/api/controller/MetricController.java @@ -31,7 +31,6 @@ import io.swagger.annotations.ApiOperation; import org.springframework.http.MediaType; import org.springframework.web.bind.annotation.*; - import javax.validation.constraints.NotNull; import java.util.*; import java.util.stream.Collectors; @@ -158,7 +157,8 @@ public Object getExpectedTypeList(@PathVariable("type") String type) { afterFilterSet = expectedValueList.stream() .map(it ->it.replace("local_", "") .replace("spark_","") - .replace("livy_","")) + .replace("livy_","") + .replace("flink_","")) .collect(Collectors.toSet()); List items = new ArrayList<>(); diff --git a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionDashboardParam.java b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionDashboardParam.java index 2d582fc87..85d8057d9 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionDashboardParam.java +++ b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionDashboardParam.java @@ -1,40 +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.server.api.dto.bo.job; - -import lombok.Data; - -import javax.validation.constraints.NotNull; - -@Data -@NotNull(message = "JobExecutionPageParam cannot be null") -public class JobExecutionDashboardParam { - - private Long datasourceId; - - private String metricType; - - private String schemaName; - - private String tableName; - - private String columnName; - - private String startTime; - - private String endTime; -} +/* + * 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.server.api.dto.bo.job; + +import lombok.Data; + +import javax.validation.constraints.NotNull; + +@Data +@NotNull(message = "JobExecutionPageParam cannot be null") +public class JobExecutionDashboardParam { + + private Long datasourceId; + + private String metricType; + + private String schemaName; + + private String tableName; + + private String columnName; + + private String startTime; + + private String endTime; +} diff --git a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionPageParam.java b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionPageParam.java index ad051507b..51013d2af 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionPageParam.java +++ b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobExecutionPageParam.java @@ -1,56 +1,56 @@ -/* - * 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.server.api.dto.bo.job; - -import lombok.Data; -import javax.validation.constraints.NotNull; - -@Data -@NotNull(message = "JobExecutionPageParam cannot be null") -public class JobExecutionPageParam { - - private Long datasourceId; - - private Integer status; - - private String searchVal; - - private Long jobId; - - private String metricType; - - private String schemaName; - - private String tableName; - - private String columnName; - - private String startTime; - - private String endTime; - - private Integer pageNumber; - - private Integer pageSize; - - private String schemaSearch; - - private String tableSearch; - - private String columnSearch; - -} +/* + * 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.server.api.dto.bo.job; + +import lombok.Data; +import javax.validation.constraints.NotNull; + +@Data +@NotNull(message = "JobExecutionPageParam cannot be null") +public class JobExecutionPageParam { + + private Long datasourceId; + + private Integer status; + + private String searchVal; + + private Long jobId; + + private String metricType; + + private String schemaName; + + private String tableName; + + private String columnName; + + private String startTime; + + private String endTime; + + private Integer pageNumber; + + private Integer pageSize; + + private String schemaSearch; + + private String tableSearch; + + private String columnSearch; + +} diff --git a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobQualityReportDashboardParam.java b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobQualityReportDashboardParam.java index f5616205e..c811f6921 100644 --- a/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobQualityReportDashboardParam.java +++ b/datavines-server/src/main/java/io/datavines/server/api/dto/bo/job/JobQualityReportDashboardParam.java @@ -1,43 +1,43 @@ -/* - * 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.server.api.dto.bo.job; - -import lombok.Data; - -import javax.validation.constraints.NotNull; - -@Data -@NotNull(message = "JobQualityReportDashboardParam cannot be null") -public class JobQualityReportDashboardParam { - - @NotNull(message = "datasourceId can not be null") - private Long datasourceId; - - private String schemaName; - - private String tableName; - - private String startTime; - - private String endTime; - - private String reportDate; - - private int pageNumber; - - private int pageSize; -} +/* + * 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.server.api.dto.bo.job; + +import lombok.Data; + +import javax.validation.constraints.NotNull; + +@Data +@NotNull(message = "JobQualityReportDashboardParam cannot be null") +public class JobQualityReportDashboardParam { + + @NotNull(message = "datasourceId can not be null") + private Long datasourceId; + + private String schemaName; + + private String tableName; + + private String startTime; + + private String endTime; + + private String reportDate; + + private int pageNumber; + + private int pageSize; +} diff --git a/datavines-ui/Editor/components/FlinkConfig/index.tsx b/datavines-ui/Editor/components/FlinkConfig/index.tsx new file mode 100644 index 000000000..8ffb8e8e4 --- /dev/null +++ b/datavines-ui/Editor/components/FlinkConfig/index.tsx @@ -0,0 +1,159 @@ +import React, { useState } from 'react'; +import { Form, Input, Radio, InputNumber, Row, Col } from 'antd'; +import { useIntl } from 'react-intl'; + +// Flink部署模式选项 +const getFlinkDeployModes = (intl: any) => [ + { label: intl.formatMessage({ id: 'dv_flink_deploy_mode_local' }), value: 'local' }, + { label: intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_session' }), value: 'yarn-session' }, + { label: intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_per_job' }), value: 'yarn-per-job' }, + { label: intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_application' }), value: 'yarn-application' }, +]; + +interface FlinkConfig { + deployMode: string; + taskManagerCount: number; + taskManagerMemory: string; + jobManagerMemory: string; + parallelism: number; + jobName: string; + yarnQueue: string; + others: string; +} + +interface FlinkConfigProps { + onChange?: (config: any) => void; + initialValues?: any; + engineType?: string; +} + +const FlinkConfig: React.FC = ({ onChange, initialValues = {}, engineType = 'flink_single_table' }) => { + const intl = useIntl(); + const [form] = Form.useForm(); + const [deployMode, setDeployMode] = useState(initialValues.deployMode || 'local'); + + const handleValuesChange = (changedValues: any, allValues: any) => { + if (changedValues.deployMode) { + setDeployMode(changedValues.deployMode); + } + + // 构建配置对象 + const config = { + env: { + deployMode: allValues.deployMode, + taskManagerCount: allValues.taskManagerCount, + taskManagerMemory: allValues.taskManagerMemory + 'G', + jobManagerMemory: allValues.jobManagerMemory + 'G', + parallelism: allValues.parallelism, + jobName: allValues.jobName, + yarnQueue: allValues.yarnQueue, + others: allValues.others + } + }; + + onChange?.(config); + }; + + const deployModes = getFlinkDeployModes(intl); + + return ( +
+ + + {deployModes.map(mode => ( + + {mode.label} + + ))} + + + + {deployMode !== 'local' && ( + <> + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + )} +
+ ); +}; + +export default FlinkConfig; diff --git a/datavines-ui/Editor/components/JobConfig/index.tsx b/datavines-ui/Editor/components/JobConfig/index.tsx new file mode 100644 index 000000000..f7d3a0544 --- /dev/null +++ b/datavines-ui/Editor/components/JobConfig/index.tsx @@ -0,0 +1,75 @@ +import React, { useState } from 'react'; +import { Form, Select } from 'antd'; +import { useIntl } from 'react-intl'; +import FlinkConfig from '../FlinkConfig'; + +const { Option } = Select; + +// 执行引擎选项 +const ENGINE_TYPES = [ + { label: 'Flink', value: 'flink_single_table' } +]; + +interface JobConfigProps { + onChange?: (config: any) => void; + initialValues?: any; +} + +const JobConfig: React.FC = ({ onChange, initialValues = {} }) => { + const intl = useIntl(); + const [form] = Form.useForm(); + const [currentEngineType, setCurrentEngineType] = useState(initialValues.engineType || 'flink_single_table'); + + const handleEngineTypeChange = (type: string) => { + setCurrentEngineType(type); + onChange?.({ + ...initialValues, + engineType: type, + engineConfig: {} + }); + }; + + const handleConfigChange = (config: any) => { + onChange?.({ + ...initialValues, + engineType: currentEngineType, + engineConfig: config + }); + }; + + return ( +
+
+ + + +
+ + {currentEngineType === 'flink_single_table' && ( + + )} +
+ ); +}; + +export default JobConfig; diff --git a/datavines-ui/Editor/components/MetricModal/ActuatorConfigure/index.tsx b/datavines-ui/Editor/components/MetricModal/ActuatorConfigure/index.tsx index a99547db0..2c6d3d668 100644 --- a/datavines-ui/Editor/components/MetricModal/ActuatorConfigure/index.tsx +++ b/datavines-ui/Editor/components/MetricModal/ActuatorConfigure/index.tsx @@ -35,7 +35,13 @@ const Index = ({ form, detail }: InnerProps) => { parameter = JSON.parse(engineParameter); } form.setFieldsValue({ - deployMode: parameter.deployMode ?? 'cluster', + deployMode: parameter.deployMode ?? 'local', + taskManagerCount: parameter.taskManagerCount ?? 2, + taskManagerMemory: parameter.taskManagerMemory ?? '2G', + jobManagerMemory: parameter.jobManagerMemory ?? '1G', + parallelism: parameter.parallelism ?? 1, + jobName: parameter.jobName ?? '', + yarnQueue: parameter.yarnQueue ?? '', driverCores: parameter.driverCores ?? 1, driverMemory: parameter.driverMemory ?? '512M', numExecutors: parameter.numExecutors ?? 2, @@ -128,6 +134,95 @@ const Index = ({ form, detail }: InnerProps) => { ); + const renderFlink = () => ( + <> + + + {intl.formatMessage({ id: 'dv_flink_deploy_mode_local' })} + {intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_session' })} + {intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_per_job' })} + {intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_application' })} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + ); return ( <Row gutter={30}> @@ -150,10 +245,13 @@ const Index = ({ form, detail }: InnerProps) => { <Form.Item noStyle dependencies={['engineType']}> {() => { const value = form.getFieldValue('engineType'); - if (value !== 'spark' && value !== 'livy') { - return null; + if (value === 'spark' || value === 'livy') { + return renderSpark(); + } + if (value === 'flink') { + return renderFlink(); } - return renderSpark(); + return null; }} </Form.Item> diff --git a/datavines-ui/Editor/components/MetricModal/RunEvnironment/index.tsx b/datavines-ui/Editor/components/MetricModal/RunEvnironment/index.tsx index 12890671c..f7a2fccae 100644 --- a/datavines-ui/Editor/components/MetricModal/RunEvnironment/index.tsx +++ b/datavines-ui/Editor/components/MetricModal/RunEvnironment/index.tsx @@ -123,10 +123,10 @@ const Index = ({ form, id, detail }: InnerProps) => { <Form.Item noStyle dependencies={['engineType']}> {() => { const value = form.getFieldValue('engineType'); - if (value !== 'spark' && value !== 'livy') { - return null; + if (value === 'spark' || value === 'livy' || value === 'flink') { + return render(); } - return render(); + return null; }} </Form.Item> ); diff --git a/datavines-ui/Editor/components/MetricModal/type.ts b/datavines-ui/Editor/components/MetricModal/type.ts index 805f85e95..f94cff3bd 100644 --- a/datavines-ui/Editor/components/MetricModal/type.ts +++ b/datavines-ui/Editor/components/MetricModal/type.ts @@ -43,14 +43,23 @@ export type TParameterItem = { } export type TEngineParameter = { - programType:string, // JAVA - deployMode:string, + programType: string, // JAVA + deployMode: string, driverCores: number, driverMemory: string, numExecutors: number, - executorMemory:string, + executorMemory: string, executorCores: number, others: string, + parallelism?: number, + jobName?: string, + yarnQueue?: string, + tenantCode?: string, + env?: string, + engineType?: string, + taskManagerCount?: number, + taskManagerMemory?: string, + jobManagerMemory?: string, } export type TDetail = null | { diff --git a/datavines-ui/Editor/locale/en_US.ts b/datavines-ui/Editor/locale/en_US.ts index 4e46d1ab6..944bc78b5 100644 --- a/datavines-ui/Editor/locale/en_US.ts +++ b/datavines-ui/Editor/locale/en_US.ts @@ -35,8 +35,11 @@ export default { dv_metric_actuator_executor_numbers: 'Number of executors', dv_metric_actuator_executor_memory: 'Number of executor memory', dv_metric_actuator_executor_cores: 'Number of executor cores', - dv_metric_actuator_executor_options: 'Options', - dv_metric_linux_user: 'Linux user', + dv_metric_actuator_executor_options: 'Options Parameters', + dv_metric_actuator_parallelism: 'Parallelism', + dv_metric_actuator_job_name: 'Job Name', + dv_metric_actuator_yarn_queue: 'Yarn Queue', + dv_metric_linux_user: 'Linux User', dv_metric_create_time: 'Create time', dv_metric_update_time: 'Update time', dv_metric_name: 'Name', diff --git a/datavines-ui/Editor/locale/zh_CN.ts b/datavines-ui/Editor/locale/zh_CN.ts index 2477389ac..51e257e05 100644 --- a/datavines-ui/Editor/locale/zh_CN.ts +++ b/datavines-ui/Editor/locale/zh_CN.ts @@ -36,6 +36,9 @@ export default { dv_metric_actuator_executor_memory: 'Executor内存数', dv_metric_actuator_executor_cores: 'Executor核心数', dv_metric_actuator_executor_options: '选项参数', + dv_metric_actuator_parallelism: '并行度', + dv_metric_actuator_job_name: '作业名称', + dv_metric_actuator_yarn_queue: 'Yarn队列', dv_metric_linux_user: 'Linux用户', dv_metric_create_time: '创建时间', dv_metric_update_time: '更新时间', diff --git a/datavines-ui/src/locale/en_US.ts b/datavines-ui/src/locale/en_US.ts index 8b2da650d..23a11bcac 100644 --- a/datavines-ui/src/locale/en_US.ts +++ b/datavines-ui/src/locale/en_US.ts @@ -292,6 +292,7 @@ export default { job_log_refresh: 'Refresh', job_log_download: 'Download', job_log_fullScreen: 'FullScreen', + dv_task_manager_count: 'Task Manager Count', error_create_btn: 'Create error data store', error_table_store_name: 'Storage Name', @@ -308,8 +309,9 @@ export default { profile_schedule: 'Profile Schedule', config_title: 'Config Management', - config_var_key: 'Config Key', - config_var_value: 'Config Value', + config_var_key: 'Variable Key', + config_var_value: 'Variable Value', + config_type: 'Configuration Type', create_config: 'Create Config', token_title: 'Token Management', @@ -329,4 +331,20 @@ export default { next_ten_cron_run_times: 'Next ten cron run times', view_future_execute_plan: 'view future execute plan', test_send: 'test send', + + dv_deploy_mode: 'Deploy Mode', + dv_deploy_mode_required: 'Please select deploy mode', + dv_flink_home: 'Flink Home Path', + dv_flink_home_required: 'Please enter Flink home path', + dv_jobmanager_memory: 'JobManager Memory (MB)', + dv_jobmanager_memory_required: 'Please enter JobManager memory size', + dv_taskmanager_memory: 'TaskManager Memory (MB)', + dv_taskmanager_memory_required: 'Please enter TaskManager memory size', + dv_flink_deploy_mode_local: 'Local Mode', + dv_flink_deploy_mode_yarn_session: 'Yarn Session Mode', + dv_flink_deploy_mode_yarn_per_job: 'Yarn Per-Job Mode', + dv_flink_deploy_mode_yarn_application: 'Yarn Application Mode', + dv_deploy_mode_cluster: 'Cluster Mode', + dv_deploy_mode_yarn: 'Yarn Mode', + dv_deploy_mode_local: 'Local Mode' }; diff --git a/datavines-ui/src/locale/zh_CN.ts b/datavines-ui/src/locale/zh_CN.ts index 4b9f06a69..ed9482c71 100644 --- a/datavines-ui/src/locale/zh_CN.ts +++ b/datavines-ui/src/locale/zh_CN.ts @@ -297,6 +297,9 @@ export default { error_title: '存储管理', user_title: '用户管理', + + dv_task_manager_count: 'Task Manager数量', + label_title: '标签分类', label_list: '标签列表', label_add_category: '新增标签分类', @@ -306,8 +309,9 @@ export default { profile_schedule: '数据概览调度配置', config_title: '参数管理', - config_var_key: '参数名', - config_var_value: '参数值', + config_var_key: '变量键', + config_var_value: '变量值', + config_type: '配置类型', create_config: '创建参数', token_title: '令牌管理', @@ -327,4 +331,20 @@ export default { next_ten_cron_run_times: '未来十次执行时间', view_future_execute_plan: '查看未来执行计划', test_send: '测试发送', + + dv_deploy_mode: '部署模式', + dv_deploy_mode_required: '请选择部署模式', + dv_flink_home: 'Flink安装路径', + dv_flink_home_required: '请输入Flink安装路径', + dv_jobmanager_memory: 'JobManager内存 (MB)', + dv_jobmanager_memory_required: '请输入JobManager内存大小', + dv_taskmanager_memory: 'TaskManager内存 (MB)', + dv_taskmanager_memory_required: '请输入TaskManager内存大小', + dv_flink_deploy_mode_local: '本地模式', + dv_flink_deploy_mode_yarn_session: 'Yarn Session模式', + dv_flink_deploy_mode_yarn_per_job: 'Yarn Per-Job模式', + dv_flink_deploy_mode_yarn_application: 'Yarn Application模式', + dv_deploy_mode_cluster: '集群模式', + dv_deploy_mode_yarn: 'Yarn模式', + dv_deploy_mode_local: '本地模式' }; diff --git a/datavines-ui/src/type/config.ts b/datavines-ui/src/type/config.ts index 99b164665..0deef55fc 100644 --- a/datavines-ui/src/type/config.ts +++ b/datavines-ui/src/type/config.ts @@ -5,5 +5,6 @@ export type TConfigTableItem = { updateTime?: string, updater?: string, varValue?: string, + type?: 'flink' | 'spark', + flinkConfig?: any, } - diff --git a/datavines-ui/src/view/Main/Config/CreateConfig.tsx b/datavines-ui/src/view/Main/Config/CreateConfig.tsx index e484a7584..bd60406d3 100644 --- a/datavines-ui/src/view/Main/Config/CreateConfig.tsx +++ b/datavines-ui/src/view/Main/Config/CreateConfig.tsx @@ -1,6 +1,6 @@ import React, { useRef, useState, useImperativeHandle } from 'react'; import { - Input, ModalProps, Form, FormInstance, message, + Input, ModalProps, Form, FormInstance, message, Select, } from 'antd'; import { useIntl } from 'react-intl'; import { @@ -9,6 +9,7 @@ import { import { $http } from '@/http'; import { useSelector } from '@/store'; import { TConfigTableItem } from "@/type/config"; +import { FlinkConfiguration } from './FlinkConfiguration'; type InnerProps = { form: FormInstance, @@ -50,6 +51,32 @@ export const CreateConfigComponent = ({ form, detail, innerRef }: InnerProps) => ], widget: <Input autoComplete="off" />, }, + { + label: intl.formatMessage({ id: 'config_type' }), + name: 'type', + initialValue: detail?.type || 'flink', + rules: [ + { + required: true, + message: intl.formatMessage({ id: 'common_required_tip' }), + }, + ], + widget: ( + <Select> + <Select.Option value="flink">Flink</Select.Option> + <Select.Option value="spark">Spark</Select.Option> + </Select> + ), + }, + { + name: 'flinkConfig', + shouldUpdate: true, + noStyle: true, + children: ({ getFieldValue }: { getFieldValue: (field: string) => string }) => { + const type = getFieldValue('type'); + return type === 'flink' ? <FlinkConfiguration form={form} detail={detail?.flinkConfig} /> : null; + }, + }, ], }; useImperativeHandle(innerRef, () => ({ diff --git a/datavines-ui/src/view/Main/Config/FlinkConfiguration.tsx b/datavines-ui/src/view/Main/Config/FlinkConfiguration.tsx new file mode 100644 index 000000000..4d6f00b08 --- /dev/null +++ b/datavines-ui/src/view/Main/Config/FlinkConfiguration.tsx @@ -0,0 +1,84 @@ +import React from 'react'; +import { Form, Input, Select, FormInstance } from 'antd'; +import { useIntl } from 'react-intl'; +import { FormRender, IFormRender } from '@/common'; + +const { Option } = Select; + +type InnerProps = { + form: FormInstance, + detail?: any +} + +export const FlinkConfiguration = ({ form, detail }: InnerProps) => { + const intl = useIntl(); + + const schema: IFormRender = { + name: 'flink-config-form', + layout: 'vertical', + formItemProps: { + style: { marginBottom: 10 }, + }, + meta: [ + { + label: intl.formatMessage({ id: 'dv_deploy_mode' }), + name: 'deployMode', + initialValue: detail?.deployMode, + rules: [ + { + required: true, + message: intl.formatMessage({ id: 'dv_deploy_mode_required' }), + }, + ], + widget: ( + <Select> + <Option value="local">{intl.formatMessage({ id: 'dv_flink_deploy_mode_local' })}</Option> + <Option value="yarn_session">{intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_session' })}</Option> + <Option value="yarn_per_job">{intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_per_job' })}</Option> + <Option value="yarn_application">{intl.formatMessage({ id: 'dv_flink_deploy_mode_yarn_application' })}</Option> + </Select> + ), + }, + { + label: intl.formatMessage({ id: 'dv_flink_home' }), + name: 'flinkHome', + initialValue: detail?.flinkHome, + rules: [ + { + required: true, + message: intl.formatMessage({ id: 'dv_flink_home_required' }), + }, + ], + widget: <Input autoComplete="off" />, + }, + { + label: intl.formatMessage({ id: 'dv_jobmanager_memory' }), + name: 'jobmanagerMemory', + initialValue: detail?.jobmanagerMemory, + rules: [ + { + required: true, + message: intl.formatMessage({ id: 'dv_jobmanager_memory_required' }), + }, + ], + widget: <Input autoComplete="off" type="number" />, + }, + { + label: intl.formatMessage({ id: 'dv_taskmanager_memory' }), + name: 'taskmanagerMemory', + initialValue: detail?.taskmanagerMemory, + rules: [ + { + required: true, + message: intl.formatMessage({ id: 'dv_taskmanager_memory_required' }), + }, + ], + widget: <Input autoComplete="off" type="number" />, + }, + ], + }; + + return <FormRender form={form} {...schema} />; +}; + +export default FlinkConfiguration; diff --git a/deploy/docker/Dockerfile b/deploy/docker/Dockerfile index d1180b0ca..b9660058a 100644 --- a/deploy/docker/Dockerfile +++ b/deploy/docker/Dockerfile @@ -41,4 +41,4 @@ RUN chmod +x datavines/bin/datavines-daemon.sh && sed -i 's/\r//g' datavines/bin EXPOSE 5600 -CMD ["/usr/bin/tini", "--", "datavines/bin/datavines-daemon.sh", "start_container", ""] +CMD ["/usr/bin/tini", "--", "datavines/bin/datavines-daemon.sh", "start_container", ""] \ No newline at end of file