diff --git a/.gitignore b/.gitignore index f642fdb5b1..ef3b6a8de2 100644 --- a/.gitignore +++ b/.gitignore @@ -30,7 +30,6 @@ dinky.db hs_err_pid* .idea/* !.idea/icon.svg -!.idea/vcs.xml build target/* *.iml @@ -63,4 +62,6 @@ tmp/* extends/* /.run/ -.idea \ No newline at end of file +.idea +.idea/vcs.xml +dinky-web/package-lock.json diff --git a/dinky-admin/src/main/java/org/dinky/init/FlinkHistoryServer.java b/dinky-admin/src/main/java/org/dinky/init/FlinkHistoryServer.java index 6aaf689bb4..f29966fbf3 100644 --- a/dinky-admin/src/main/java/org/dinky/init/FlinkHistoryServer.java +++ b/dinky-admin/src/main/java/org/dinky/init/FlinkHistoryServer.java @@ -19,10 +19,8 @@ package org.dinky.init; -import org.dinky.data.model.ResourcesModelEnum; -import org.dinky.data.model.S3Configuration; import org.dinky.data.model.SystemConfiguration; -import org.dinky.data.properties.OssProperties; +import org.dinky.resource.BaseResourceManager; import org.dinky.service.JobInstanceService; import org.dinky.service.SysConfigService; @@ -63,16 +61,7 @@ public FlinkHistoryServer(JobInstanceService jobInstanceService, SysConfigServic this.historyRunnable = () -> { Map flinkHistoryServerConfiguration = SystemConfiguration.getInstances().getFlinkHistoryServerConfiguration(); - if (systemConfiguration.getResourcesEnable().getValue()) { - if (systemConfiguration.getResourcesModel().getValue().equals(ResourcesModelEnum.OSS)) { - OssProperties ossProperties = systemConfiguration.getOssProperties(); - flinkHistoryServerConfiguration.put(S3Configuration.ENDPOINT, ossProperties.getEndpoint()); - flinkHistoryServerConfiguration.put(S3Configuration.ACCESS_KEY, ossProperties.getAccessKey()); - flinkHistoryServerConfiguration.put(S3Configuration.SECRET_KEY, ossProperties.getSecretKey()); - flinkHistoryServerConfiguration.put( - S3Configuration.PATH_STYLE_ACCESS, String.valueOf(ossProperties.getPathStyleAccess())); - } - } + flinkHistoryServerConfiguration.putAll(BaseResourceManager.convertFlinkResourceConfig()); HistoryServerUtil.run( (jobId) -> { diff --git a/dinky-cdc/dinky-cdc-core/src/main/java/org/dinky/cdc/convert/DataTypeConverter.java b/dinky-cdc/dinky-cdc-core/src/main/java/org/dinky/cdc/convert/DataTypeConverter.java index e12c0d6d8a..a1f80fde30 100644 --- a/dinky-cdc/dinky-cdc-core/src/main/java/org/dinky/cdc/convert/DataTypeConverter.java +++ b/dinky-cdc/dinky-cdc-core/src/main/java/org/dinky/cdc/convert/DataTypeConverter.java @@ -49,7 +49,6 @@ import java.time.LocalDateTime; import java.time.LocalTime; import java.time.ZoneId; -import java.util.Optional; import java.util.concurrent.TimeUnit; import javax.xml.bind.DatatypeConverter; @@ -117,7 +116,7 @@ public static LogicalType getLogicalType(Column column) { public static Object convertToRow(Object value, LogicalType logicalType, ZoneId timeZone) { if (Asserts.isNull(value)) { - return Optional.empty(); + return null; } switch (logicalType.getTypeRoot()) { case BOOLEAN: @@ -163,7 +162,7 @@ public static Object convertToRow(Object value, LogicalType logicalType, ZoneId public static Object convertToRowData(Object value, LogicalType logicalType, ZoneId timeZone) { if (Asserts.isNull(value)) { - return Optional.empty(); + return null; } switch (logicalType.getTypeRoot()) { case BOOLEAN: diff --git a/dinky-client/dinky-client-base/src/main/java/org/dinky/resource/BaseResourceManager.java b/dinky-client/dinky-client-base/src/main/java/org/dinky/resource/BaseResourceManager.java index af9b0e7935..e5e6d0e845 100644 --- a/dinky-client/dinky-client-base/src/main/java/org/dinky/resource/BaseResourceManager.java +++ b/dinky-client/dinky-client-base/src/main/java/org/dinky/resource/BaseResourceManager.java @@ -19,9 +19,13 @@ package org.dinky.resource; +import org.dinky.assertion.Asserts; import org.dinky.data.exception.DinkyException; +import org.dinky.data.model.ResourcesModelEnum; import org.dinky.data.model.ResourcesVO; +import org.dinky.data.model.S3Configuration; import org.dinky.data.model.SystemConfiguration; +import org.dinky.data.properties.OssProperties; import org.dinky.oss.OssTemplate; import org.dinky.resource.impl.HdfsResourceManager; import org.dinky.resource.impl.LocalResourceManager; @@ -34,7 +38,9 @@ import java.io.IOException; import java.io.InputStream; import java.nio.charset.Charset; +import java.util.HashMap; import java.util.List; +import java.util.Map; import cn.hutool.core.io.FileUtil; import cn.hutool.core.io.IoUtil; @@ -74,6 +80,9 @@ static BaseResourceManager getInstance() { } static void initResourceManager() { + if (Asserts.isNull(instances.getResourcesModel().getValue())) { + return; + } switch (instances.getResourcesModel().getValue()) { case LOCAL: Singleton.get(LocalResourceManager.class); @@ -106,6 +115,37 @@ static void initResourceManager() { } } + static Map convertFlinkResourceConfig() { + Map flinkConfig = new HashMap<>(); + if (!instances.getResourcesEnable().getValue()) { + return flinkConfig; + } + if (instances.getResourcesModel().getValue().equals(ResourcesModelEnum.OSS)) { + OssProperties ossProperties = instances.getOssProperties(); + flinkConfig.put(S3Configuration.ENDPOINT, ossProperties.getEndpoint()); + flinkConfig.put(S3Configuration.ACCESS_KEY, ossProperties.getAccessKey()); + flinkConfig.put(S3Configuration.SECRET_KEY, ossProperties.getSecretKey()); + flinkConfig.put(S3Configuration.PATH_STYLE_ACCESS, String.valueOf(ossProperties.getPathStyleAccess())); + } else if (instances.getResourcesModel().getValue().equals(ResourcesModelEnum.HDFS)) { + final Configuration configuration = new Configuration(); + Charset charset = Charset.defaultCharset(); + String coreSite = instances.getResourcesHdfsCoreSite().getValue(); + Opt.ofBlankAble(coreSite).ifPresent(x -> configuration.addResource(IoUtil.toStream(x, charset))); + String hdfsSite = instances.getResourcesHdfsHdfsSite().getValue(); + Opt.ofBlankAble(hdfsSite).ifPresent(x -> configuration.addResource(IoUtil.toStream(x, charset))); + configuration.reloadConfiguration(); + if (StrUtil.isEmpty(coreSite)) { + configuration.set( + "fs.defaultFS", instances.getResourcesHdfsDefaultFS().getValue()); + } + Map hadoopConfig = configuration.getValByRegex(".*"); + hadoopConfig.forEach((key, value) -> { + flinkConfig.put("flink.hadoop." + key, value); + }); + } + return flinkConfig; + } + default String getFilePath(String path) { return FileUtil.normalize(FileUtil.file(getBasePath(), path).toString()); } diff --git a/dinky-core/src/main/java/org/dinky/executor/Executor.java b/dinky-core/src/main/java/org/dinky/executor/Executor.java index acf43bf1d3..777918e434 100644 --- a/dinky-core/src/main/java/org/dinky/executor/Executor.java +++ b/dinky-core/src/main/java/org/dinky/executor/Executor.java @@ -31,6 +31,7 @@ import org.dinky.interceptor.FlinkInterceptor; import org.dinky.interceptor.FlinkInterceptorResult; import org.dinky.job.JobStatementPlan; +import org.dinky.resource.BaseResourceManager; import org.dinky.trans.Operations; import org.dinky.utils.KerberosUtil; @@ -169,6 +170,7 @@ private void initClassloader(DinkyClassLoader classLoader) { protected void init(DinkyClassLoader classLoader) { initClassloader(classLoader); + initFileSystem(); this.dinkyClassLoader = classLoader; Thread.currentThread().setContextClassLoader(classLoader); if (executorConfig.isValidParallelism()) { @@ -195,6 +197,10 @@ protected void init(DinkyClassLoader classLoader) { isMockTest = false; } + private void initFileSystem() { + BaseResourceManager.initResourceManager(); + } + abstract CustomTableEnvironment createCustomTableEnvironment(ClassLoader classLoader); public JobStatementPlan parseStatementIntoJobStatementPlan(String[] statements) { diff --git a/dinky-core/src/main/java/org/dinky/explainer/Explainer.java b/dinky-core/src/main/java/org/dinky/explainer/Explainer.java index 4fed7f4afd..39d86200e4 100644 --- a/dinky-core/src/main/java/org/dinky/explainer/Explainer.java +++ b/dinky-core/src/main/java/org/dinky/explainer/Explainer.java @@ -98,7 +98,20 @@ public Explainer initialize(JobConfig config, String statement) { } public JobStatementPlan parseStatements(String[] statements) { - JobStatementPlan jobStatementPlanWithUDFAndMock = new JobStatementPlan(); + JobStatementPlan jobStatementPlanWithMock = new JobStatementPlan(); + generateUDFStatement(jobStatementPlanWithMock); + + JobStatementPlan jobStatementPlan = executor.parseStatementIntoJobStatementPlan(statements); + jobStatementPlanWithMock.getJobStatementList().addAll(jobStatementPlan.getJobStatementList()); + if (!jobManager.isPlanMode() && jobManager.getConfig().isMockSinkFunction()) { + executor.setMockTest(true); + MockStatementExplainer.build(executor.getCustomTableEnvironment()) + .jobStatementPlanMock(jobStatementPlanWithMock); + } + return jobStatementPlanWithMock; + } + + private void generateUDFStatement(JobStatementPlan jobStatementPlan) { List udfStatements = new ArrayList<>(); Optional.ofNullable(jobManager.getConfig().getUdfRefer()) .ifPresent(t -> t.forEach((key, value) -> { @@ -106,16 +119,8 @@ public JobStatementPlan parseStatements(String[] statements) { udfStatements.add(sql); })); for (String udfStatement : udfStatements) { - jobStatementPlanWithUDFAndMock.addJobStatement(udfStatement, JobStatementType.DDL, SqlType.CREATE); - } - JobStatementPlan jobStatementPlan = executor.parseStatementIntoJobStatementPlan(statements); - jobStatementPlanWithUDFAndMock.getJobStatementList().addAll(jobStatementPlan.getJobStatementList()); - if (!jobManager.isPlanMode() && jobManager.getConfig().isMockSinkFunction()) { - executor.setMockTest(true); - MockStatementExplainer.build(executor.getCustomTableEnvironment()) - .jobStatementPlanMock(jobStatementPlanWithUDFAndMock); + jobStatementPlan.addJobStatement(udfStatement, JobStatementType.DDL, SqlType.CREATE); } - return jobStatementPlanWithUDFAndMock; } public List parseUDFFromStatements(String[] statements) { diff --git a/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java b/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java index 8c7c1f18e9..15cbb8e689 100644 --- a/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java +++ b/dinky-core/src/main/java/org/dinky/job/runner/JobPipelineRunner.java @@ -34,6 +34,7 @@ import org.dinky.job.Job; import org.dinky.job.JobConfig; import org.dinky.job.JobManager; +import org.dinky.trans.parse.ExecuteJarParseStrategy; import org.dinky.utils.FlinkStreamEnvironmentUtil; import org.dinky.utils.LogUtil; import org.dinky.utils.SqlUtil; @@ -66,6 +67,11 @@ public JobPipelineRunner(JobManager jobManager) { @Override public void run(JobStatement jobStatement) throws Exception { + if (ExecuteJarParseStrategy.INSTANCE.match(jobStatement.getStatement())) { + JobJarRunner jobJarRunner = new JobJarRunner(jobManager); + jobJarRunner.run(jobStatement); + return; + } statements.add(jobStatement); tableResult = jobManager.getExecutor().executeSql(jobStatement.getStatement()); if (statements.size() == 1) { @@ -83,6 +89,10 @@ public void run(JobStatement jobStatement) throws Exception { @Override public SqlExplainResult explain(JobStatement jobStatement) { + if (ExecuteJarParseStrategy.INSTANCE.match(jobStatement.getStatement())) { + JobJarRunner jobJarRunner = new JobJarRunner(jobManager); + return jobJarRunner.explain(jobStatement); + } SqlExplainResult.Builder resultBuilder = SqlExplainResult.Builder.newBuilder(); statements.add(jobStatement); // pipeline job execute to generate stream graph. @@ -131,6 +141,10 @@ public SqlExplainResult explain(JobStatement jobStatement) { @Override public StreamGraph getStreamGraph(JobStatement jobStatement) { + if (ExecuteJarParseStrategy.INSTANCE.match(jobStatement.getStatement())) { + JobJarRunner jobJarRunner = new JobJarRunner(jobManager); + return jobJarRunner.getStreamGraph(jobStatement); + } statements.add(jobStatement); // pipeline job execute to generate stream graph. jobManager.getExecutor().executeSql(jobStatement.getStatement()); @@ -144,6 +158,10 @@ public StreamGraph getStreamGraph(JobStatement jobStatement) { @Override public JobPlanInfo getJobPlanInfo(JobStatement jobStatement) { + if (ExecuteJarParseStrategy.INSTANCE.match(jobStatement.getStatement())) { + JobJarRunner jobJarRunner = new JobJarRunner(jobManager); + return jobJarRunner.getJobPlanInfo(jobStatement); + } statements.add(jobStatement); // pipeline job execute to generate stream graph. jobManager.getExecutor().executeSql(jobStatement.getStatement()); diff --git a/dinky-function/src/main/java/org/dinky/function/FlinkUDFDiscover.java b/dinky-function/src/main/java/org/dinky/function/FlinkUDFDiscover.java index 2b9ab4f595..89eaf67f19 100644 --- a/dinky-function/src/main/java/org/dinky/function/FlinkUDFDiscover.java +++ b/dinky-function/src/main/java/org/dinky/function/FlinkUDFDiscover.java @@ -20,18 +20,21 @@ package org.dinky.function; import org.dinky.function.data.model.UDF; +import org.dinky.function.util.Reflections; import org.apache.commons.collections.CollectionUtils; import org.apache.flink.table.catalog.FunctionLanguage; import org.apache.flink.table.functions.UserDefinedFunction; +import java.io.File; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Set; import java.util.stream.Collectors; -import org.reflections.Reflections; import org.reflections.scanners.Scanners; -import org.reflections.util.ClasspathHelper; import org.reflections.util.ConfigurationBuilder; public class FlinkUDFDiscover { @@ -42,9 +45,28 @@ public static List getCustomStaticUDFs() { if (CollectionUtils.isNotEmpty(JAVA_STATIC_UDF_LIST)) { return JAVA_STATIC_UDF_LIST; } + Collection urls = new ArrayList<>(); + String javaClassPath = System.getProperty("java.class.path"); + if (javaClassPath != null) { + for (String path : javaClassPath.split(File.pathSeparator)) { + if (path.contains("/*")) { + continue; + } + if (!path.contains("extends") && !path.contains("customJar")) { + continue; + } - Reflections reflections = - new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath())); + try { + urls.add(new File(path).toURI().toURL()); + } catch (Exception e) { + if (Reflections.log != null) { + Reflections.log.warn("Could not get URL", e); + } + } + } + } + + Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(urls)); Set> operations = reflections.get(Scanners.SubTypes.of(UserDefinedFunction.class).asClass()); return operations.stream() diff --git a/dinky-function/src/main/java/org/dinky/function/util/Reflections.java b/dinky-function/src/main/java/org/dinky/function/util/Reflections.java new file mode 100644 index 0000000000..872b656e80 --- /dev/null +++ b/dinky-function/src/main/java/org/dinky/function/util/Reflections.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 org.dinky.function.util; + +import static java.lang.String.format; +import static org.reflections.scanners.Scanners.SubTypes; +import static org.reflections.scanners.Scanners.TypesAnnotated; + +import java.io.BufferedInputStream; +import java.io.DataInputStream; +import java.lang.annotation.Annotation; +import java.net.URL; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Predicate; +import java.util.stream.Collectors; + +import javassist.bytecode.ClassFile; + +import javax.annotation.Nullable; + +import org.reflections.Configuration; +import org.reflections.ReflectionUtils; +import org.reflections.ReflectionsException; +import org.reflections.Store; +import org.reflections.scanners.Scanner; +import org.reflections.scanners.Scanners; +import org.reflections.util.NameHelper; +import org.reflections.util.QueryFunction; +import org.reflections.vfs.Vfs; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Reflections one-stop-shop object + */ +public class Reflections implements NameHelper { + public static final Logger log = LoggerFactory.getLogger(Reflections.class); + + protected final transient Configuration configuration; + protected final Store store; + + /** + * constructs Reflections instance and scan according to the given {@link Configuration} + *

it is preferred to use {@link org.reflections.util.ConfigurationBuilder}

{@code new Reflections(new ConfigurationBuilder()...)}
+ */ + public Reflections(Configuration configuration) { + this.configuration = configuration; + Map>> storeMap = scan(); + if (configuration.shouldExpandSuperTypes()) { + expandSuperTypes(storeMap.get(SubTypes.index()), storeMap.get(TypesAnnotated.index())); + } + store = new Store(storeMap); + } + + protected Map>> scan() { + long start = System.currentTimeMillis(); + Map>> collect = configuration.getScanners().stream() + .map(Scanner::index) + .distinct() + .collect(Collectors.toMap(s -> s, s -> Collections.synchronizedSet(new HashSet<>()))); + Set urls = configuration.getUrls(); + + (configuration.isParallel() ? urls.stream().parallel() : urls.stream()).forEach(url -> { + Vfs.Dir dir = null; + try { + dir = Vfs.fromURL(url); + for (Vfs.File file : dir.getFiles()) { + if (doFilter(file, configuration.getInputsFilter())) { + ClassFile classFile = null; + for (Scanner scanner : configuration.getScanners()) { + try { + if (doFilter(file, scanner::acceptsInput)) { + List> entries = scanner.scan(file); + if (entries == null) { + if (classFile == null) classFile = getClassFile(file); + entries = scanner.scan(classFile); + } + if (entries != null) + collect.get(scanner.index()).addAll(entries); + } + } catch (Exception e) { + if (log != null) + log.trace( + "could not scan file {} with scanner {}", + file.getRelativePath(), + scanner.getClass().getSimpleName(), + e); + } + } + } + } + } catch (Exception e) { + // nothing to do. + } finally { + if (dir != null) dir.close(); + } + }); + + // merge + Map>> storeMap = collect.entrySet().stream() + .collect(Collectors.toMap(Map.Entry::getKey, entry -> entry.getValue().stream() + .filter(e -> e.getKey() != null) + .collect(Collectors.groupingBy( + Map.Entry::getKey, + HashMap::new, + Collectors.mapping(Map.Entry::getValue, Collectors.toSet()))))); + if (log != null) { + int keys = 0, values = 0; + for (Map> map : storeMap.values()) { + keys += map.size(); + values += map.values().stream().mapToLong(Set::size).sum(); + } + log.info(format( + "Reflections took %d ms to scan %d urls, producing %d keys and %d values", + System.currentTimeMillis() - start, urls.size(), keys, values)); + } + return storeMap; + } + + private boolean doFilter(Vfs.File file, @Nullable Predicate predicate) { + String path = file.getRelativePath(); + String fqn = path.replace('/', '.'); + return predicate == null || predicate.test(path) || predicate.test(fqn); + } + + private ClassFile getClassFile(Vfs.File file) { + try (DataInputStream dis = new DataInputStream(new BufferedInputStream(file.openInputStream()))) { + return new ClassFile(dis); + } catch (Exception e) { + throw new ReflectionsException("could not create class object from file " + file.getRelativePath(), e); + } + } + + /** + * expand super types after scanning, for super types that were not scanned. + *
this is helpful for finding the transitive closure without scanning all 3rd party dependencies. + *

+ * for example, for classes A,B,C where A supertype of B, B supertype of C (A -> B -> C): + *
    + *
  • if scanning C resulted in B (B->C in store), but A was not scanned (although A is a supertype of B) - then getSubTypes(A) will not return C
  • + *
  • if expanding supertypes, B will be expanded with A (A->B in store) - then getSubTypes(A) will return C
  • + *
+ */ + public void expandSuperTypes(Map> subTypesStore, Map> typesAnnotatedStore) { + if (subTypesStore == null || subTypesStore.isEmpty()) return; + Set keys = new LinkedHashSet<>(subTypesStore.keySet()); + keys.removeAll( + subTypesStore.values().stream().flatMap(Collection::stream).collect(Collectors.toSet())); + keys.remove("java.lang.Object"); + for (String key : keys) { + Class type = forClass(key, loaders()); + if (type != null) { + expandSupertypes(subTypesStore, typesAnnotatedStore, key, type); + } + } + } + + private void expandSupertypes( + Map> subTypesStore, + Map> typesAnnotatedStore, + String key, + Class type) { + Set typeAnnotations = ReflectionUtils.getAnnotations(type); + if (typesAnnotatedStore != null && !typeAnnotations.isEmpty()) { + String typeName = type.getName(); + for (Annotation typeAnnotation : typeAnnotations) { + String annotationName = typeAnnotation.annotationType().getName(); + typesAnnotatedStore + .computeIfAbsent(annotationName, s -> new HashSet<>()) + .add(typeName); + } + } + for (Class supertype : ReflectionUtils.getSuperTypes(type)) { + String supertypeName = supertype.getName(); + if (subTypesStore.containsKey(supertypeName)) { + subTypesStore.get(supertypeName).add(key); + } else { + subTypesStore + .computeIfAbsent(supertypeName, s -> new HashSet<>()) + .add(key); + expandSupertypes(subTypesStore, typesAnnotatedStore, supertypeName, supertype); + } + } + } + + /** + * apply {@link QueryFunction} on {@link Store} + *
{@code Set ts = get(query)}
+ *

use {@link Scanners} and {@link ReflectionUtils} query functions, such as: + *

{@code
+     * Set annotated = get(Scanners.TypesAnnotated.with(A.class))
+     * Set> subtypes = get(Scanners.SubTypes.of(B.class).asClass())
+     * Set methods = get(ReflectionUtils.Methods.of(B.class))
+     * }
+ */ + public Set get(QueryFunction query) { + return query.apply(store); + } + + ClassLoader[] loaders() { + return configuration.getClassLoaders(); + } +} diff --git a/docs/download/dinky-1.2.0.md b/docs/download/dinky-1.2.0.md index 203759e05f..406ab0e33d 100644 --- a/docs/download/dinky-1.2.0.md +++ b/docs/download/dinky-1.2.0.md @@ -3,15 +3,15 @@ sidebar_position: 78 title: 1.2.0 release --- -| Dinky 版本 | Flink 版本 | 二进制程序 | Source | -|----------|----------|---------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------| -| 1.2.0 | 1.14 | [dinky-release-1.14-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.14-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.15 | [dinky-release-1.15-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.15-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.16 | [dinky-release-1.16-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.16-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.17 | [dinky-release-1.17-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.17-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.18 | [dinky-release-1.18-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.18-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.19 | [dinky-release-1.19-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.19-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | -| 1.2.0 | 1.20 | [dinky-release-1.20-1.2.0.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.19-1.2.0.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| Dinky 版本 | Flink 版本 | 二进制程序 | Source | +|----------|----------|-------------------------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------| +| 1.2.0 | 1.14 | [dinky-release-1.14-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.14-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.15 | [dinky-release-1.15-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.15-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.16 | [dinky-release-1.16-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.16-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.17 | [dinky-release-1.17-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.17-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.18 | [dinky-release-1.18-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.18-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.19 | [dinky-release-1.19-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.19-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | +| 1.2.0 | 1.20 | [dinky-release-1.20-1.2.0-rc4.tar.gz](https://github.com/DataLinkDC/dinky/releases/download/v1.2.0/dinky-release-1.19-1.2.0-rc4.tar.gz) | [Source code (zip)](https://github.com/DataLinkDC/dinky/archive/refs/tags/v1.2.0.zip) | ## Dinky-1.2.0 发行说明 diff --git a/pom.xml b/pom.xml index 784e2287fe..cd73757a55 100644 --- a/pom.xml +++ b/pom.xml @@ -110,7 +110,7 @@ UTF-8 2.5.0 0.10.2 - 1.2.0-rc2 + 1.2.0-rc4 1.37.0 2.12.10